You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/04/11 09:31:26 UTC

[GitHub] [ozone] guihecheng opened a new pull request, #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

guihecheng opened a new pull request, #3292:
URL: https://github.com/apache/ozone/pull/3292

   ## What changes were proposed in this pull request?
   
   Per-disk DB location management.
   More descriptions about the db location could be found in the JIRA below.
   Here are some descriptions of the 3 separated commits:
   - Add some LayoutFeature definitions but not the whole non-rolling upgrade related stuff, so we could check if we have to init the per-disk db instances.
   - A new `StorageVolume` type `DbVolume` for optional dedicated SSDs for db instances to speed up meta operations.
   - Format db instances on DN first register and load db instances on DN startup.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6541
   
   ## How was this patch tested?
   
   New UTs.
   


-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r853948538


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/DatanodeStoreCache.java:
##########
@@ -58,7 +58,17 @@ public RawDB getDB(String containerDBPath) {
   }
 
   public void removeDB(String containerDBPath) {
-    datanodeStoreMap.remove(containerDBPath);
+    RawDB db = datanodeStoreMap.remove(containerDBPath);
+    if (db == null) {
+      LOG.debug("DB {} already removed", containerDBPath);
+      return;
+    }
+
+    try {
+      db.getStore().stop();
+    } catch (Exception e) {
+      LOG.warn("Stop DatanodeStore: {} failed", containerDBPath, e);

Review Comment:
   Use error level. 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on PR #3292:
URL: https://github.com/apache/ozone/pull/3292#issuecomment-1106110028

   The last patch LGTM, +1.  
   
   Thanks @guihecheng  for the contribution.


-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r850322484


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   For 2, failed volume check doesn't belong here, it is an async check and DbVolumes are handled as other volumes, please check `StorageVolumeChecker.java`.  And I still want a clear choice between (A)"throw & out" (B)"continue"?
   
   For 3.1, let me explain `hddsFiles.length == 2`:  one file is for VERSION; the other file is either clusterIDDir or scmIDDir, SCM HA checks whether the Dir is (A)clusterIDDir or (B)scmIDDir. If A, do nothing because it means SCM HA is finalized; If B, upgrade this volume for SCM HA by linking. But no matter A or B, we have an IDDir which implies that this HddsVolume is formatted on the first DN register, and we won't do db format again for it neither on DN register nor DN restart. Please check the code inside this function for the words above `VersionedDatanodeFeatures.ScmHA.upgradeVolumeIfNeeded`.
   So here I mean I don't actually depends on SCM HA to check whether we have to format db instance or not, clear?
   
   For 3.2, after 3.1, I think you should know about the meaning of `hddsFiles.length`. And let me say more about when we have the chance to create a db instance:
   - Along with HddsVolume format on DN first register, so only when we have`hddsFiles.length == 1` which means that we only got a VERSION file on the HddsVolume, so we have to create the clusterIDDir/scmIDDir only on the first DN register.
   - Along with load on DN restart, I think this is where you actually suspect that we may have duplicate db instance creation. But I should say that, there is a check `!storageIdDir.exists()` at line 350 of file `HddsVolumeUtil.java` that checks if the subdirectry with `StorageID` exists, we only load the db instance when this exists. And for now I think I should have a stricter check for the `container.db` directory exactly, then we prevent from creating db instances on all pathes. Make sense?



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on PR #3292:
URL: https://github.com/apache/ozone/pull/3292#issuecomment-1096555755

   @ChenSammi Thanks for your comments, I'll update soon.


-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848061519


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java:
##########
@@ -133,6 +138,16 @@ public OzoneContainer(
     volumeSet.setFailedVolumeListener(this::handleVolumeFailures);
     metaVolumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(), conf,
         context, VolumeType.META_VOLUME, volumeChecker);
+    if (VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()
+        .equals(OzoneConsts.SCHEMA_V3)) {
+      dbVolumeSet = HddsServerUtil.getDatanodeDbDirs(conf).isEmpty() ? null :

Review Comment:
   1.  Shutdown this dbVolumeSet in #stop function.
   2. #getNodeReport should handle this new dbVolumeSet. 
   3. need a new scanner for dbVolumeSet.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi merged pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi merged PR #3292:
URL: https://github.com/apache/ozone/pull/3292


-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r851042482


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   We'll have a dedicated patch for non-rolling upgrade at last and we could define non-rolling upgrade hooks to help create the db instances upon `FinalizeUpgrade`.
   We could implement those hooks by referring to the hooks defined for SCM HA.
   (BTW, in the internal version we disabled the non-rolling upgrade feature, so the db instance logic is a bit different from the code present here.)



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r851136043


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   OK,  sounds good. 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848265280


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {
+      return format(clusterID);
+    }
+
+    if (!getStorageDir().exists()) {
+      // Not formatted yet
+      return true;
+    }
+
+    File[] storageDirs = getStorageDir().listFiles(File::isDirectory);
+    if (storageDirs == null) {
+      LOG.error("IO error for the db volume {}, skipped loading",
+          getStorageDir());
+      return false;

Review Comment:
   OK, I'll follow the handling of those in `HddsVolume`.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {
+      return format(clusterID);
+    }
+
+    if (!getStorageDir().exists()) {
+      // Not formatted yet
+      return true;
+    }
+
+    File[] storageDirs = getStorageDir().listFiles(File::isDirectory);
+    if (storageDirs == null) {
+      LOG.error("IO error for the db volume {}, skipped loading",
+          getStorageDir());
+      return false;

Review Comment:
   OK, I'll follow the handling of those in `HddsVolume`.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848144983


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {

Review Comment:
   Don't get the point of this check. Would you do a further explanation? 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849021981


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   For storage, data integrity and security is very important.  So we better be conservative and fail the code fast if something happens which requires admin's notice and interference. So please throw out the exception here and other places. 
   
   Besides, default value of failedDbVolumeTolerated is -1, which is too optimistic for a storage system. We should change the default value to 0 for all failedVolumeTolerated. But we can do it in a follow up PR.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849289279


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   > 1. For the failed volume tolerated value, read this thread [HDDS-5219. Limit number of bad volumes by dfs.datanode.failed.volumes.tolerated #2243](https://github.com/apache/ozone/pull/2243) which you've participant in, there's a discussion about the default 0 or -1. You could send a patch to fix it overall with the above reasons later after you have decided about the best default value, but not in this thread.
   
   It can be done in another PR. I will take care of that. 
   
   > 
   >     2. For the "throw or continue" problem, first please provide an answer to my question first, what is the desired behavior(with no extra ssds) in your mind and the reason, then we try to reach an agreement.
   
   If rocksdb is on the data volume. We follow the data volume rule. If the rocksdb load is failed,  it means this data volume fails.  If the failed data volume exceeds the tolerance,  then DN fails.  Is this reasonable? 
   
   > 
   >     3. For the new question you raised, suppose there are 2 DbVolumes and one of them is down, the db instances on that one is unreadable and all related HddsVolumes are unreadable. This scenario equals to that half of the HddsVolumes are down. This is not a very extreme case, even the whole DN is down, data integrity and security is not fatally harmed. But there won't be any new db instances created on the remaining dbVolume as you said since db instances are only created when we try to format an HddsVolume on the DN registration, please recheck.
   
   If I remember correctly, DN will register to SCM on every startup, which means if one DbVolume failed, all impacted HddsVolume will have no DbVolume and ParentStorageDir set, then it need create a new RocksDB in function #volumeCheck. Right? 
   
   
   
   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on PR #3292:
URL: https://github.com/apache/ozone/pull/3292#issuecomment-1102167334

   Hi @ChenSammi , I've made a new push just now.
   Comments address check list:
   
   - [ ] ozone-default description
   - [ ] shutdown dbVolumeSet
   - [ ] nodeReport for dbVolumeSet
   - [ ] VERSION file logic extracted into a common place StorageVolume, so DbVolume now has VERSION file
   - [ ] dbParentDir is only set at the end of the function
   - [ ] a config switch for schemaV3 with a helper function to check it
   - [ ] tests check that no duplicate db instance will be created on dbVolume failures.
   - [ ] Some renames like formatXXX -> createXXX
   
   Apart from the comments above, I've make some refactors to keep related things together.


-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r853941497


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_NAME;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final Set<String> hddsVolumeIDs;

Review Comment:
   Can we change this Set to a Map, hold both the storageID and db path? 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on PR #3292:
URL: https://github.com/apache/ozone/pull/3292#issuecomment-1094810028

   @ChenSammi @nandakumar131 PTAL~


-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848144983


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {

Review Comment:
   Don't get the point of this check. Would you do a further explanation? 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r851018486


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   Another case I just thought about is upgrading. If we enable this feature on an existing cluster,  HddsFiles.length will be 2 in such case, so which piece of code will help us to create this rocksDB per disk instance? 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849190788


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   It depends on how many disk failure is tolerated.  For example, in HDFS,  default disk failure tolerated is 0, which means as long as one disk fails, DN cannot start up.  Admin may change the configuration based on his/her estimation of disk failure rate.  That's acceptable and the risk is on Admin.  So "at least one disk volume" is not acceptable, the risk is too high. 
   
   Think about this case, all HDDS volumes function well while one of the two configured dbVolumes is down at DN startup.  How will the impacted HDDS volume behave? Create a new RocksDB instance on the remaining dbVolume and then go on to provide service?  I think we should persist this HDDS volume to RocksDB instance relation in HDDS version file.   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849289279


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   > 1. For the failed volume tolerated value, read this thread [HDDS-5219. Limit number of bad volumes by dfs.datanode.failed.volumes.tolerated #2243](https://github.com/apache/ozone/pull/2243) which you've participant in, there's a discussion about the default 0 or -1. You could send a patch to fix it overall with the above reasons later after you have decided about the best default value, but not in this thread.
   
   It can be done in another PR. I will take care of that. 
   
   > 
   >     2. For the "throw or continue" problem, first please provide an answer to my question first, what is the desired behavior(with no extra ssds) in your mind and the reason, then we try to reach an agreement.
   
   If rocksdb is on the data volume. We follow the data volume rule. If the rocksdb load is failed,  it means this data volume fails.  If the failed data volume exceeds the tolerance,  then DN fails.  Is this reasonable? 
   
   > 
   >     3. For the new question you raised, suppose there are 2 DbVolumes and one of them is down, the db instances on that one is unreadable and all related HddsVolumes are unreadable. This scenario equals to that half of the HddsVolumes are down. This is not a very extreme case, even the whole DN is down, data integrity and security is not fatally harmed. But there won't be any new db instances created on the remaining dbVolume as you said since db instances are only created when we try to format an HddsVolume on the DN registration, please recheck.
   
   If I remember correctly, DN will register to SCM on every startup, which means if one DbVolume failed, all impacted HddsVolume will have no DbVolume and ParentStorageDir set, then it will create a new RocksDB in function #volumeCheck. Right? 
   
   
   
   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848134204


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {
+      return format(clusterID);
+    }
+
+    if (!getStorageDir().exists()) {
+      // Not formatted yet
+      return true;
+    }
+
+    File[] storageDirs = getStorageDir().listFiles(File::isDirectory);
+    if (storageDirs == null) {
+      LOG.error("IO error for the db volume {}, skipped loading",
+          getStorageDir());
+      return false;

Review Comment:
   Please throw out Exception in all false case.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r853961191


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -71,167 +61,65 @@ public static String getHddsRoot(String volumeRoot) {
   }
 
   /**
-   * Returns storageID if it is valid. Throws an exception otherwise.
-   */
-  @VisibleForTesting
-  public static String getStorageID(Properties props, File versionFile)
-      throws InconsistentStorageStateException {
-    return getProperty(props, OzoneConsts.STORAGE_ID, versionFile);
-  }
-
-  /**
-   * Returns clusterID if it is valid. It should match the clusterID from the
-   * Datanode. Throws an exception otherwise.
-   */
-  @VisibleForTesting
-  public static String getClusterID(Properties props, File versionFile,
-      String clusterID) throws InconsistentStorageStateException {
-    String cid = getProperty(props, OzoneConsts.CLUSTER_ID, versionFile);
-
-    if (clusterID == null) {
-      return cid;
-    }
-    if (!clusterID.equals(cid)) {
-      throw new InconsistentStorageStateException("Mismatched " +
-          "ClusterIDs. Version File : " + versionFile + " has clusterID: " +
-          cid + " and Datanode has clusterID: " + clusterID);
-    }
-    return cid;
-  }
-
-  /**
-   * Returns datanodeUuid if it is valid. It should match the UUID of the
-   * Datanode. Throws an exception otherwise.
-   */
-  @VisibleForTesting
-  public static String getDatanodeUUID(Properties props, File versionFile,
-      String datanodeUuid)
-      throws InconsistentStorageStateException {
-    String datanodeID = getProperty(props, OzoneConsts.DATANODE_UUID,
-        versionFile);
-
-    if (datanodeUuid != null && !datanodeUuid.equals(datanodeID)) {
-      throw new InconsistentStorageStateException("Mismatched " +
-          "DatanodeUUIDs. Version File : " + versionFile + " has datanodeUuid: "
-          + datanodeID + " and Datanode has datanodeUuid: " + datanodeUuid);
-    }
-    return datanodeID;
-  }
-
-  /**
-   * Returns creationTime if it is valid. Throws an exception otherwise.
+   * Initialize db instance, rocksdb will load the existing instance
+   * if present and format a new one if not.
+   * @param containerDBPath
+   * @param conf
+   * @throws IOException
    */
-  @VisibleForTesting
-  public static long getCreationTime(Properties props, File versionFile)
-      throws InconsistentStorageStateException {
-    String cTimeStr = getProperty(props, OzoneConsts.CTIME, versionFile);
-
-    long cTime = Long.parseLong(cTimeStr);
-    long currentTime = Time.now();
-    if (cTime > currentTime || cTime < 0) {
-      throw new InconsistentStorageStateException("Invalid Creation time in " +
-          "Version File : " + versionFile + " - " + cTime + ". Current system" +
-          " time is " + currentTime);
-    }
-    return cTime;
+  public static void initPerDiskDBStore(String containerDBPath,
+      ConfigurationSource conf) throws IOException {
+    DatanodeStore store = BlockUtils.getUncachedDatanodeStore(containerDBPath,
+        OzoneConsts.SCHEMA_V3, conf, false);
+    BlockUtils.addDB(store, containerDBPath, conf, OzoneConsts.SCHEMA_V3);
   }
 
   /**
-   * Returns layOutVersion if it is valid. Throws an exception otherwise.
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param logger
    */
-  @VisibleForTesting
-  public static int getLayOutVersion(Properties props, File versionFile) throws
-      InconsistentStorageStateException {
-    String lvStr = getProperty(props, OzoneConsts.LAYOUTVERSION, versionFile);
-
-    int lv = Integer.parseInt(lvStr);
-    if (HDDSVolumeLayoutVersion.getLatestVersion().getVersion() != lv) {
-      throw new InconsistentStorageStateException("Invalid layOutVersion. " +
-          "Version file has layOutVersion as " + lv + " and latest Datanode " +
-          "layOutVersion is " +
-          HDDSVolumeLayoutVersion.getLatestVersion().getVersion());
-    }
-    return lv;
-  }
-
-  public static String getProperty(
-      Properties props, String propName, File
-      versionFile
-  )
-      throws InconsistentStorageStateException {
-    String value = props.getProperty(propName);
-    if (StringUtils.isBlank(value)) {
-      throw new InconsistentStorageStateException("Invalid " + propName +
-          ". Version File : " + versionFile + " has null or empty " + propName);
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    for (HddsVolume volume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      try {
+        volume.loadDbStore();
+      } catch (IOException e) {
+        onFailure(volume);

Review Comment:
   Shall we just fail the volume, like "hddsVolumeSet.failVolume(volume.getStorageDir().getPath());"?



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r854171000


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/StorageVolumeUtil.java:
##########
@@ -48,4 +65,189 @@ public static List<HddsVolume> getHddsVolumesList(
     return volumes.stream().
         map(v -> (HddsVolume) v).collect(Collectors.toList());
   }
+
+  public static List<DbVolume> getDbVolumesList(
+      List<StorageVolume> volumes) {
+    return volumes.stream().
+        map(v -> (DbVolume) v).collect(Collectors.toList());
+  }
+
+  public static File getVersionFile(File rootDir) {
+    return new File(rootDir, VERSION_FILE);
+  }
+
+  public static String generateUuid() {
+    return STORAGE_ID_PREFIX + UUID.randomUUID();
+  }
+
+  /**
+   * Returns storageID if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getStorageID(Properties props, File versionFile)
+      throws InconsistentStorageStateException {
+    return getProperty(props, OzoneConsts.STORAGE_ID, versionFile);
+  }
+
+  /**
+   * Returns clusterID if it is valid. It should match the clusterID from the
+   * Datanode. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getClusterID(Properties props, File versionFile,
+      String clusterID) throws InconsistentStorageStateException {
+    String cid = getProperty(props, OzoneConsts.CLUSTER_ID, versionFile);
+
+    if (clusterID == null) {
+      return cid;
+    }
+    if (!clusterID.equals(cid)) {
+      throw new InconsistentStorageStateException("Mismatched " +
+          "ClusterIDs. Version File : " + versionFile + " has clusterID: " +
+          cid + " and Datanode has clusterID: " + clusterID);
+    }
+    return cid;
+  }
+
+  /**
+   * Returns datanodeUuid if it is valid. It should match the UUID of the
+   * Datanode. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getDatanodeUUID(Properties props, File versionFile,
+      String datanodeUuid)
+      throws InconsistentStorageStateException {
+    String datanodeID = getProperty(props, OzoneConsts.DATANODE_UUID,
+        versionFile);
+
+    if (datanodeUuid != null && !datanodeUuid.equals(datanodeID)) {
+      throw new InconsistentStorageStateException("Mismatched " +
+          "DatanodeUUIDs. Version File : " + versionFile + " has datanodeUuid: "
+          + datanodeID + " and Datanode has datanodeUuid: " + datanodeUuid);
+    }
+    return datanodeID;
+  }
+
+  /**
+   * Returns creationTime if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static long getCreationTime(Properties props, File versionFile)
+      throws InconsistentStorageStateException {
+    String cTimeStr = getProperty(props, OzoneConsts.CTIME, versionFile);
+
+    long cTime = Long.parseLong(cTimeStr);
+    long currentTime = Time.now();
+    if (cTime > currentTime || cTime < 0) {
+      throw new InconsistentStorageStateException("Invalid Creation time in " +
+          "Version File : " + versionFile + " - " + cTime + ". Current system" +
+          " time is " + currentTime);
+    }
+    return cTime;
+  }
+
+  /**
+   * Returns layOutVersion if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static int getLayOutVersion(Properties props, File versionFile) throws
+      InconsistentStorageStateException {
+    String lvStr = getProperty(props, OzoneConsts.LAYOUTVERSION, versionFile);
+
+    int lv = Integer.parseInt(lvStr);
+    if (HDDSVolumeLayoutVersion.getLatestVersion().getVersion() != lv) {
+      throw new InconsistentStorageStateException("Invalid layOutVersion. " +
+          "Version file has layOutVersion as " + lv + " and latest Datanode " +
+          "layOutVersion is " +
+          HDDSVolumeLayoutVersion.getLatestVersion().getVersion());
+    }
+    return lv;
+  }
+
+  public static String getProperty(
+      Properties props, String propName, File
+      versionFile
+  )
+      throws InconsistentStorageStateException {
+    String value = props.getProperty(propName);
+    if (StringUtils.isBlank(value)) {
+      throw new InconsistentStorageStateException("Invalid " + propName +
+          ". Version File : " + versionFile + " has null or empty " + propName);
+    }
+    return value;
+  }
+
+  /**
+   * Check Volume is in consistent state or not.
+   * Prior to SCM HA, volumes used the format {@code <volume>/hdds/<scm-id>}.
+   * Post SCM HA, new volumes will use the format {@code <volume>/hdds/<cluster
+   * -id>}.
+   * Existing volumes using SCM ID would have been reformatted to have {@code
+   * <volume>/hdds/<cluster-id>} as a symlink pointing to {@code <volume
+   * >/hdds/<scm-id>}.
+   *
+   * @param volume
+   * @param scmId
+   * @param clusterId
+   * @param conf
+   * @param logger
+   * @param dbVolumeSet
+   * @return true - if volume is in consistent state, otherwise false.
+   */
+  public static boolean checkVolume(StorageVolume volume, String scmId,
+      String clusterId, ConfigurationSource conf, Logger logger,
+      MutableVolumeSet dbVolumeSet) {
+    File hddsRoot = volume.getStorageDir();

Review Comment:
   OK, than I shall rename it to `volumeRoot`.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r854176697


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -71,167 +61,65 @@ public static String getHddsRoot(String volumeRoot) {
   }
 
   /**
-   * Returns storageID if it is valid. Throws an exception otherwise.
-   */
-  @VisibleForTesting
-  public static String getStorageID(Properties props, File versionFile)
-      throws InconsistentStorageStateException {
-    return getProperty(props, OzoneConsts.STORAGE_ID, versionFile);
-  }
-
-  /**
-   * Returns clusterID if it is valid. It should match the clusterID from the
-   * Datanode. Throws an exception otherwise.
-   */
-  @VisibleForTesting
-  public static String getClusterID(Properties props, File versionFile,
-      String clusterID) throws InconsistentStorageStateException {
-    String cid = getProperty(props, OzoneConsts.CLUSTER_ID, versionFile);
-
-    if (clusterID == null) {
-      return cid;
-    }
-    if (!clusterID.equals(cid)) {
-      throw new InconsistentStorageStateException("Mismatched " +
-          "ClusterIDs. Version File : " + versionFile + " has clusterID: " +
-          cid + " and Datanode has clusterID: " + clusterID);
-    }
-    return cid;
-  }
-
-  /**
-   * Returns datanodeUuid if it is valid. It should match the UUID of the
-   * Datanode. Throws an exception otherwise.
-   */
-  @VisibleForTesting
-  public static String getDatanodeUUID(Properties props, File versionFile,
-      String datanodeUuid)
-      throws InconsistentStorageStateException {
-    String datanodeID = getProperty(props, OzoneConsts.DATANODE_UUID,
-        versionFile);
-
-    if (datanodeUuid != null && !datanodeUuid.equals(datanodeID)) {
-      throw new InconsistentStorageStateException("Mismatched " +
-          "DatanodeUUIDs. Version File : " + versionFile + " has datanodeUuid: "
-          + datanodeID + " and Datanode has datanodeUuid: " + datanodeUuid);
-    }
-    return datanodeID;
-  }
-
-  /**
-   * Returns creationTime if it is valid. Throws an exception otherwise.
+   * Initialize db instance, rocksdb will load the existing instance
+   * if present and format a new one if not.
+   * @param containerDBPath
+   * @param conf
+   * @throws IOException
    */
-  @VisibleForTesting
-  public static long getCreationTime(Properties props, File versionFile)
-      throws InconsistentStorageStateException {
-    String cTimeStr = getProperty(props, OzoneConsts.CTIME, versionFile);
-
-    long cTime = Long.parseLong(cTimeStr);
-    long currentTime = Time.now();
-    if (cTime > currentTime || cTime < 0) {
-      throw new InconsistentStorageStateException("Invalid Creation time in " +
-          "Version File : " + versionFile + " - " + cTime + ". Current system" +
-          " time is " + currentTime);
-    }
-    return cTime;
+  public static void initPerDiskDBStore(String containerDBPath,
+      ConfigurationSource conf) throws IOException {
+    DatanodeStore store = BlockUtils.getUncachedDatanodeStore(containerDBPath,
+        OzoneConsts.SCHEMA_V3, conf, false);
+    BlockUtils.addDB(store, containerDBPath, conf, OzoneConsts.SCHEMA_V3);
   }
 
   /**
-   * Returns layOutVersion if it is valid. Throws an exception otherwise.
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param logger
    */
-  @VisibleForTesting
-  public static int getLayOutVersion(Properties props, File versionFile) throws
-      InconsistentStorageStateException {
-    String lvStr = getProperty(props, OzoneConsts.LAYOUTVERSION, versionFile);
-
-    int lv = Integer.parseInt(lvStr);
-    if (HDDSVolumeLayoutVersion.getLatestVersion().getVersion() != lv) {
-      throw new InconsistentStorageStateException("Invalid layOutVersion. " +
-          "Version file has layOutVersion as " + lv + " and latest Datanode " +
-          "layOutVersion is " +
-          HDDSVolumeLayoutVersion.getLatestVersion().getVersion());
-    }
-    return lv;
-  }
-
-  public static String getProperty(
-      Properties props, String propName, File
-      versionFile
-  )
-      throws InconsistentStorageStateException {
-    String value = props.getProperty(propName);
-    if (StringUtils.isBlank(value)) {
-      throw new InconsistentStorageStateException("Invalid " + propName +
-          ". Version File : " + versionFile + " has null or empty " + propName);
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    for (HddsVolume volume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      try {
+        volume.loadDbStore();
+      } catch (IOException e) {
+        onFailure(volume);

Review Comment:
   Nope, `onFailure` and `failVolume` are different.
   Here the db load failed, it may be caused by a failed disk or other reasons like: not enough memory, rocksdb bugs, etc.
   So `onFailure` checks whether this volume is bad or not asynchronously. Only with the `onFailure` call we could trigger a potential "max tolerated volumes reached" event, failVolume will not do.
   And we can't just `failVolume` here because there may be containers of old schemas(v1, v2) on this volume, we should keep them readable.
   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848061519


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java:
##########
@@ -133,6 +138,16 @@ public OzoneContainer(
     volumeSet.setFailedVolumeListener(this::handleVolumeFailures);
     metaVolumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(), conf,
         context, VolumeType.META_VOLUME, volumeChecker);
+    if (VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()
+        .equals(OzoneConsts.SCHEMA_V3)) {
+      dbVolumeSet = HddsServerUtil.getDatanodeDbDirs(conf).isEmpty() ? null :

Review Comment:
   Shutdown this dbVolumeSet in #stop function.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r850486869


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   And for the 2 DbVolumes with 1 failed case, first I should make it clear that the case you raised is already prevented with current code(plus the stricter check that I proposed), because those HddsVolumes with failed db instances on the failed dbVolume still have their clusterIDDir created, so no new db instance will be able to create since the HddsVolume is already `checkVolume`d and never again.
   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r850272728


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   
   >     2. The behavior you stated sounds reasonable to me(I'll check bad volume if db load failed in the next push), so as I understand, we should continue rather than throw&out when we hit a bad volume. Agreed?
   
   After all the discussions, we should add the failed hddsVolume to failed volume list, then check that if failed volumes exceed the limit, right?  
   
   > 
   >     3. Let me clarify it a bit. A DbVolume is only a place holder for potential db instances for HddsVolumes, it is formatted when DN register happens(only for now, I'll change it to format on DN startup), before all HddsVolumes. A db instance is formatted for each HddsVolume when `checkVolume` is called for the first time(after all DbVolumes are formatted). Check my comment below about the condition check `hddsFiles.length == 1`. Upon the first register of DN -> SCM, DN get a clusterID back and create a directory(e.g. /data1/hdds/CID-) for each HddsVolume, we only create a db instance together with the clusterID directory. On the 2nd, 3rd,... DN restart and registers, `checkVolume` will found that `hddsFiles.length == 2`, so no redundant db instance will be created. Does it make sense?
   
   Here are two cases, 
   1. RocksDB on HddsVolume.  Your explanation makes sense. But, current hddsFiles.length == 2 logic is SCM HA upgrade fix.  If you depends on hddsFiles.length 2 to detect whether RocksDB created or not, you need to distinguish these two different cases. 
   2. RocksDB on dbVolume. In the case, RocksDB existence is irrelevant with hddsFiles.length, right? 
   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848267407


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {
+      return format(clusterID);
+    }
+
+    if (!getStorageDir().exists()) {
+      // Not formatted yet
+      return true;

Review Comment:
   Yes, as I understand, generally, here we should have similar volume state lifecycle as `HddsVolume` does.
   I think I'll try to extract the VolumeState and VERSION file stuff into a common place such as `StorageVolume`,
   then both `HddsVolume` and `DbVolume` could have the same management.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849224867


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   1. For the failed volume tolerated value, read this thread https://github.com/apache/ozone/pull/2243 which you've participant in, there's a discussion about the default 0 or -1. You could send a patch to fix it overall with the above reasons later after you have decided about the best default value, but not in this thread.
   2. For the "throw or continue" problem, first please provide an answer to my question first, what is the desired behavior(with no extra ssds) in your mind and the reason, then we try to reach an agreement.
   3. For the new question you raised, suppose there are 2 DbVolumes and one of them is down, the db instances on that one is unreadable and all related HddsVolumes are unreadable. This scenario equals to that half of the HddsVolumes are down. This is not a very extreme case, even the whole DN is down, data integrity and security is not fatally harmed. But there won't be any new db instances created on the remaining dbVolume as you said since db instances are only created when we try to format an HddsVolume on the DN registration, please recheck.
   
   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849083579


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   Just a simple question here: say we have 100 disks for a DN, on extra SSDs, so db instances are on the same disk as data. If a disk failure causes the db init fails, do we continue for other disks or just throw and out?
   I don't think we will stop DN startup on a disk failure for now with per-container db instances since disk failure is common, it does not really hurt data integrity or security for a distributed system.
   
   The failedDbVolumeTolerated defaults to -1 means we'll have at least one dbVolume if configured any, please check `hasEnoughVolumes#hasEnoughVolumes`.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r853950366


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/StorageVolumeUtil.java:
##########
@@ -48,4 +65,189 @@ public static List<HddsVolume> getHddsVolumesList(
     return volumes.stream().
         map(v -> (HddsVolume) v).collect(Collectors.toList());
   }
+
+  public static List<DbVolume> getDbVolumesList(
+      List<StorageVolume> volumes) {
+    return volumes.stream().
+        map(v -> (DbVolume) v).collect(Collectors.toList());
+  }
+
+  public static File getVersionFile(File rootDir) {
+    return new File(rootDir, VERSION_FILE);
+  }
+
+  public static String generateUuid() {
+    return STORAGE_ID_PREFIX + UUID.randomUUID();
+  }
+
+  /**
+   * Returns storageID if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getStorageID(Properties props, File versionFile)
+      throws InconsistentStorageStateException {
+    return getProperty(props, OzoneConsts.STORAGE_ID, versionFile);
+  }
+
+  /**
+   * Returns clusterID if it is valid. It should match the clusterID from the
+   * Datanode. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getClusterID(Properties props, File versionFile,
+      String clusterID) throws InconsistentStorageStateException {
+    String cid = getProperty(props, OzoneConsts.CLUSTER_ID, versionFile);
+
+    if (clusterID == null) {
+      return cid;
+    }
+    if (!clusterID.equals(cid)) {
+      throw new InconsistentStorageStateException("Mismatched " +
+          "ClusterIDs. Version File : " + versionFile + " has clusterID: " +
+          cid + " and Datanode has clusterID: " + clusterID);
+    }
+    return cid;
+  }
+
+  /**
+   * Returns datanodeUuid if it is valid. It should match the UUID of the
+   * Datanode. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getDatanodeUUID(Properties props, File versionFile,
+      String datanodeUuid)
+      throws InconsistentStorageStateException {
+    String datanodeID = getProperty(props, OzoneConsts.DATANODE_UUID,
+        versionFile);
+
+    if (datanodeUuid != null && !datanodeUuid.equals(datanodeID)) {
+      throw new InconsistentStorageStateException("Mismatched " +
+          "DatanodeUUIDs. Version File : " + versionFile + " has datanodeUuid: "
+          + datanodeID + " and Datanode has datanodeUuid: " + datanodeUuid);
+    }
+    return datanodeID;
+  }
+
+  /**
+   * Returns creationTime if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static long getCreationTime(Properties props, File versionFile)
+      throws InconsistentStorageStateException {
+    String cTimeStr = getProperty(props, OzoneConsts.CTIME, versionFile);
+
+    long cTime = Long.parseLong(cTimeStr);
+    long currentTime = Time.now();
+    if (cTime > currentTime || cTime < 0) {
+      throw new InconsistentStorageStateException("Invalid Creation time in " +
+          "Version File : " + versionFile + " - " + cTime + ". Current system" +
+          " time is " + currentTime);
+    }
+    return cTime;
+  }
+
+  /**
+   * Returns layOutVersion if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static int getLayOutVersion(Properties props, File versionFile) throws
+      InconsistentStorageStateException {
+    String lvStr = getProperty(props, OzoneConsts.LAYOUTVERSION, versionFile);
+
+    int lv = Integer.parseInt(lvStr);
+    if (HDDSVolumeLayoutVersion.getLatestVersion().getVersion() != lv) {
+      throw new InconsistentStorageStateException("Invalid layOutVersion. " +
+          "Version file has layOutVersion as " + lv + " and latest Datanode " +
+          "layOutVersion is " +
+          HDDSVolumeLayoutVersion.getLatestVersion().getVersion());
+    }
+    return lv;
+  }
+
+  public static String getProperty(
+      Properties props, String propName, File
+      versionFile
+  )
+      throws InconsistentStorageStateException {
+    String value = props.getProperty(propName);
+    if (StringUtils.isBlank(value)) {
+      throw new InconsistentStorageStateException("Invalid " + propName +
+          ". Version File : " + versionFile + " has null or empty " + propName);
+    }
+    return value;
+  }
+
+  /**
+   * Check Volume is in consistent state or not.
+   * Prior to SCM HA, volumes used the format {@code <volume>/hdds/<scm-id>}.
+   * Post SCM HA, new volumes will use the format {@code <volume>/hdds/<cluster
+   * -id>}.
+   * Existing volumes using SCM ID would have been reformatted to have {@code
+   * <volume>/hdds/<cluster-id>} as a symlink pointing to {@code <volume
+   * >/hdds/<scm-id>}.
+   *
+   * @param volume
+   * @param scmId
+   * @param clusterId
+   * @param conf
+   * @param logger
+   * @param dbVolumeSet
+   * @return true - if volume is in consistent state, otherwise false.
+   */
+  public static boolean checkVolume(StorageVolume volume, String scmId,
+      String clusterId, ConfigurationSource conf, Logger logger,
+      MutableVolumeSet dbVolumeSet) {
+    File hddsRoot = volume.getStorageDir();

Review Comment:
   Should use a more generic name other than "hdds" in this function now. 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r854170456


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/DatanodeStoreCache.java:
##########
@@ -58,7 +58,17 @@ public RawDB getDB(String containerDBPath) {
   }
 
   public void removeDB(String containerDBPath) {
-    datanodeStoreMap.remove(containerDBPath);
+    RawDB db = datanodeStoreMap.remove(containerDBPath);
+    if (db == null) {
+      LOG.debug("DB {} already removed", containerDBPath);
+      return;
+    }
+
+    try {
+      db.getStore().stop();
+    } catch (Exception e) {
+      LOG.warn("Stop DatanodeStore: {} failed", containerDBPath, e);

Review Comment:
   Fine.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r850325366


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   And if you have time, please check the descriptions in the JIRA above, there you could see that layout picture of db locations, and you'll be more clear about this.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   And if you have time, please check the descriptions in the JIRA above, there you could see that layout picture of db locations, and you'll be more clear about my words.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r850481576


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   OK, then I think I can take (B) for the answer to my question and keep the "continue" way if there is a db load failure now if you are not against it. Of course I'll rewrite the confusing `loadAllHddsVolumeDbStore ` into a cleaner function like:
   `
   void loadAllHddsVolumeDbStore() {
       try {
          loadHddsVolumeDbStore();
       } catch (IOException e) {
           LOG.error(...);
           continue;
       }
   }
   `
   In this way, I think we'll are be satisfied, there is Exception thrown and we'll continue the load for other good volumes.
   
   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849190788


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   It depends on how many disk failure is tolerated.  For example, in HDFS,  default disk failure tolerated is 0, which means as long as one disk fails, DN cannot start up.  Admin may change the configuration based on his/her estimation of disk failure rate.  That's acceptable and the risk is on Admin.  So "at least one disk volume" is not acceptable, the risk is too high. 
   
   Think about this case, all HDDS volumes function well while one of the two configured dbVolumes is down at DN startup.  How will the impacted HDDS volume behave? Create a new RocksDB instance on the remaining dbVolume and then go on to provide service?  I think we should persist this HDDS volume to RocksDB instance relation on HDDS version file.   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849021981


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   For storage, data integrity and security is very important.  So we better be conservative and fail the code fast if something happens which requires admin's notice and interference. So please throw out the exception here. 
   
   Besides, default value of failedDbVolumeTolerated is -1, which is too optimistic for a storage system. We should change the default value to 0 for all failedVolumeTolerated. But we can do it in a follow up PR.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848233524


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;
+      }
+
+      File storageIdDir = new File(clusterIdDir, volume.getStorageID());
+      if (!storageIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "storageID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              storageIdDir.getAbsolutePath());
+        }
+        continue;
+      }
+      volume.setDbParentDir(storageIdDir);

Review Comment:
   Move this statement to the end of this function.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848224189


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   Please throw exception instead.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848284837


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;
+      }
+
+      File storageIdDir = new File(clusterIdDir, volume.getStorageID());
+      if (!storageIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "storageID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              storageIdDir.getAbsolutePath());
+        }
+        continue;
+      }
+      volume.setDbParentDir(storageIdDir);
+
+      String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+          .getAbsolutePath();
+      try {
+        initPerDiskDBStore(containerDBPath, conf);
+      } catch (IOException e) {
+        if (logger != null) {
+          logger.error("Can't load db instance under path {} for volume {}",
+              containerDBPath, volume.getStorageDir().getAbsolutePath());

Review Comment:
   Well, if you don't really like the way that we log an error and go on for other instances, I could split this big functions into smaller ones and do throw inside and (catch & go on) outside.
   But generally, we are going to make best effort, or we'll miss some good db instances.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;
+      }
+
+      File storageIdDir = new File(clusterIdDir, volume.getStorageID());
+      if (!storageIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "storageID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              storageIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   ditto



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848285235


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java:
##########
@@ -129,4 +136,31 @@ public EndpointStateMachine.EndPointStates call() throws Exception {
     }
     return rpcEndPoint.getState();
   }
+
+  private void formatDbVolumesIfNeeded(String scmId, String clusterId) {
+    if (!VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()
+        .equals(OzoneConsts.SCHEMA_V3)) {
+      return;
+    }
+
+    MutableVolumeSet dbVolumeSet = ozoneContainer.getDbVolumeSet();
+
+    if (dbVolumeSet != null) {
+      dbVolumeSet.writeLock();
+      try {
+        List<DbVolume> dbVolumeList = StorageVolumeUtil
+            .getDbVolumesList(dbVolumeSet.getVolumesList());
+
+        dbVolumeList.parallelStream().forEach(dbVolume -> {
+          String id = VersionedDatanodeFeatures.ScmHA
+              .chooseContainerPathID(configuration, scmId, clusterId);

Review Comment:
   OK, makes sense.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r847989229


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -144,6 +144,20 @@
       tagged explicitly.
     </description>
   </property>
+  <property>
+    <name>hdds.datanode.container.db.dir</name>
+    <value/>
+    <tag>OZONE, CONTAINER, STORAGE, MANAGEMENT</tag>
+    <description>Determines where the per-disk rocksdb instances will be
+      stored. Defaults to empty if not specified, then rocksdb instances

Review Comment:
   Defaults to empty if not specified  -> This setting is optional.  If unspecified



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848219703


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {

Review Comment:
   Will check volume.getDbVolume() not null be more straightforward? 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848249078


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java:
##########
@@ -129,4 +136,31 @@ public EndpointStateMachine.EndPointStates call() throws Exception {
     }
     return rpcEndPoint.getState();
   }
+
+  private void formatDbVolumesIfNeeded(String scmId, String clusterId) {
+    if (!VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()
+        .equals(OzoneConsts.SCHEMA_V3)) {
+      return;
+    }
+
+    MutableVolumeSet dbVolumeSet = ozoneContainer.getDbVolumeSet();
+
+    if (dbVolumeSet != null) {
+      dbVolumeSet.writeLock();
+      try {
+        List<DbVolume> dbVolumeList = StorageVolumeUtil
+            .getDbVolumesList(dbVolumeSet.getVolumesList());
+
+        dbVolumeList.parallelStream().forEach(dbVolume -> {
+          String id = VersionedDatanodeFeatures.ScmHA
+              .chooseContainerPathID(configuration, scmId, clusterId);

Review Comment:
   Move this statement out of the stream.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848134204


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {
+      return format(clusterID);
+    }
+
+    if (!getStorageDir().exists()) {
+      // Not formatted yet
+      return true;
+    }
+
+    File[] storageDirs = getStorageDir().listFiles(File::isDirectory);
+    if (storageDirs == null) {
+      LOG.error("IO error for the db volume {}, skipped loading",
+          getStorageDir());
+      return false;

Review Comment:
   Please throw out Exception in this case.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848136136


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {
+      return format(clusterID);
+    }
+
+    if (!getStorageDir().exists()) {
+      // Not formatted yet
+      return true;

Review Comment:
   Please do the format action here. Don't delay it to the later #checkVolume function. 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on PR #3292:
URL: https://github.com/apache/ozone/pull/3292#issuecomment-1096492239

   @guihecheng, I just left some comments.  We can have a call if some comments confuse you. 


-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849083579


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   Just a simple question here: say we have 100 disks for a DN, on extra SSDs, so db instances are on the same disk as data. If a disk failure causes the db init fails, do we continue for other disks or just throw and out?
   I don't think we will stop DN startup on a disk failure for now since disk failure is common, it does not really hurt data integrity or security for a distributed system.
   
   The failedDbVolumeTolerated defaults to -1 means we'll have at least one dbVolume if configured any, please check `hasEnoughVolumes#hasEnoughVolumes`.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r850394110


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   @guihecheng , thanks for the explanation. Got the logic of  HddsFiles.length now. Just feel that the logic is not that straightforward and a little hard to understand.  So in a summary, we only create the rocksdb on a new DN's first time registration. After that, the only place where DN will load rocksDB instances is in #loadAllHddsVolumeDbStore. 
   
   Let's back to the use case that one of two dbVolumes is failed. In #mapDbVolumesToDataVolumesIfNeeded, say half of the HddsVolumes have their dbVolume loaded and set, half of them don't because of their corresponding dbVolume is failed. We need to identify this case in #mapDbVolumesToDataVolumesIfNeeded and present the impacted HddsVolume from creating a new RocksDB under its sub-directory. 
   
   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r850490012


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   At last, I think I should really add dedicated tests for the cases you raised seriously, thanks for checking it.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r850493560


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -215,6 +227,17 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
         logger.error("Unable to create ID directory {} for datanode.", idDir);
         return false;
       }
+
+      if (VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()

Review Comment:
   Yes, of course, I can't agree more with Arpit.
   It would be easy for me to add this config back to the codes.
   Having this config item also benefits the compatibility test as I tested this feature internally.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849319038


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   1. Thanks for it, I'll help review it then if able to.
   2. The behavior you stated sounds reasonable to me(I'll check bad volume if db load failed in the next push), so as I understand, we should continue rather than throw&out when we hit a bad volume. Agreed?
   3. Let me clarify it a bit. A DbVolume is only a place holder for potential db instances for HddsVolumes, it is formatted (only create a root directory and VERSION file)when DN register happens, before all HddsVolumes. A db instance is formatted for each HddsVolume when `checkVolume` is called for the first time(after all DbVolumes are formatted). Check my comment below about the condition check `hddsFiles.length == 1`. Upon the first register of DN -> SCM, DN get a clusterID back and create a directory(e.g. /data1/hdds/CID-<clusterID>) for each HddsVolume, we only create a db instance together with the clusterID directory. On the 2nd, 3rd,... DN registers, `checkVolume` will found that `hddsFiles.length == 2`, so no redundant db instance will be created. Does it make sense?



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848237688


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;
+      }
+
+      File storageIdDir = new File(clusterIdDir, volume.getStorageID());
+      if (!storageIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "storageID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              storageIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   Throw exception here.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849190788


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   It depends on how many disk failure is tolerated.  For example, in HDFS,  default disk failure tolerated is 0, which means as long as one disk fails, DN cannot start up.  Admin may change the configuration based on his/her estimation of disk failure rate.  That's acceptable and the risk is on Admin.  So "at least one disk volume" is not acceptable, the risk is too high. 
   
   Think about this case, all HDDS volumes function well while one of the two configured dbVolumes is down at DN startup.  How will the impacted HDDS volume behave? Create a new RocksDB instance on the remaining dbVolume and then go on to provide service?  I think we should persist this HDDS volume to RocksDB instance relation on HDDS
   version file.   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849319038


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   1. Thanks for it, I'll help review it then if able to.
   2. The behavior you stated sounds reasonable to me(I'll check bad volume if db load failed in the next push), so as I understand, we should continue rather than throw&out when we hit a bad volume. Agreed?
   3. Let me clarify it a bit. A DbVolume is only a place holder for potential db instances for HddsVolumes, it is formatted when DN register happens(only for now, I'll change it to format on DN startup), before all HddsVolumes. A db instance is formatted for each HddsVolume when `checkVolume` is called for the first time(after all DbVolumes are formatted). Check my comment below about the condition check `hddsFiles.length == 1`. Upon the first register of DN -> SCM, DN get a clusterID back and create a directory(e.g. /data1/hdds/CID-<clusterID>) for each HddsVolume, we only create a db instance together with the clusterID directory. On the 2nd, 3rd,... DN registers, `checkVolume` will found that `hddsFiles.length == 2`, so no redundant db instance will be created. Does it make sense?



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848264850


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java:
##########
@@ -133,6 +138,16 @@ public OzoneContainer(
     volumeSet.setFailedVolumeListener(this::handleVolumeFailures);
     metaVolumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(), conf,
         context, VolumeType.META_VOLUME, volumeChecker);
+    if (VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()
+        .equals(OzoneConsts.SCHEMA_V3)) {
+      dbVolumeSet = HddsServerUtil.getDatanodeDbDirs(conf).isEmpty() ? null :

Review Comment:
   1. Oh, yes, we should shutdown there
   2. Sure.
   3. That's a good idea, we could have a new scanner to check the db instances in another PR.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848061519


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java:
##########
@@ -133,6 +138,16 @@ public OzoneContainer(
     volumeSet.setFailedVolumeListener(this::handleVolumeFailures);
     metaVolumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(), conf,
         context, VolumeType.META_VOLUME, volumeChecker);
+    if (VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()
+        .equals(OzoneConsts.SCHEMA_V3)) {
+      dbVolumeSet = HddsServerUtil.getDatanodeDbDirs(conf).isEmpty() ? null :

Review Comment:
   1.  Shutdown this dbVolumeSet in #stop function.
   2. #getNodeReport should handle this new dbVolumeSet. 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848136136


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {
+      return format(clusterID);
+    }
+
+    if (!getStorageDir().exists()) {
+      // Not formatted yet
+      return true;

Review Comment:
   1. Please do the format action here. Don't delay it to the later #checkVolume function. 
   2. check the storageDir is a directory or not. Refer to HddsVolume #analyzeVolumeState function.
   3. need a VERSION file under the storageDir. Refer to HddsVolume#createVersionFile.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848231053


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -215,6 +227,17 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
         logger.error("Unable to create ID directory {} for datanode.", idDir);
         return false;
       }
+
+      if (VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()
+          .equals(OzoneConsts.SCHEMA_V3)) {

Review Comment:
   Add a check that hddsVolume.setDbParentDir is set to avoid duplicate rockdb initialization. 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848276030


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {

Review Comment:
   The clusterID comes from the SCM, but in all Unit tests, we don't really have a SCM, so we always have a global clusterID set and passed to `MutableVolumeSet`, so all volumes in the set will get initialized.
   But as discussed above, if we are going to do format early rather than in `checkVolume`, then we may not need this.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848280815


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -215,6 +227,17 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
         logger.error("Unable to create ID directory {} for datanode.", idDir);
         return false;
       }
+
+      if (VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()
+          .equals(OzoneConsts.SCHEMA_V3)) {

Review Comment:
   Actually we don't have to, because here we have the check above that `hddsFiles.length == 1`, so we are sure that this `HddsVolume` is fresh.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r850401111


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -215,6 +227,17 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
         logger.error("Unable to create ID directory {} for datanode.", idDir);
         return false;
       }
+
+      if (VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()

Review Comment:
   Please use a helper function to detect whether SchemaV3 is enabled. And user this helper function in all places where need a detection.  
   
   I discussed the feature with Arpit today. His suggestion is in the first release version, we disable this feature by default.  User can turn on it manually. So other than this VersionedDataFeatures check, we may need a property, which disable, or enable this feature, just like we did internally. 
    When this feature become mature enough, we deprecate this property and make the feature always enabled.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849319038


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   1. Thanks for it, I'll help review it then if able to.
   2. The behavior you stated sounds reasonable to me(I'll check bad volume if db load failed in the next push), so as I understand, we should continue rather than throw&out when we hit a bad volume. Agreed?
   3. Let me clarify it a bit. A DbVolume is only a place holder for potential db instances for HddsVolumes, it is formatted when DN register happens(only for now, I'll change it to format on DN startup), before all HddsVolumes. A db instance is formatted for each HddsVolume when `checkVolume` is called for the first time(after all DbVolumes are formatted). Check my comment below about the condition check `hddsFiles.length == 1`. Upon the first register of DN -> SCM, DN get a clusterID back and create a directory(e.g. /data1/hdds/CID-<clusterID>) for each HddsVolume, we only create a db instance together with the clusterID directory. On the 2nd, 3rd,... DN restart and registers, `checkVolume` will found that `hddsFiles.length == 2`, so no redundant db instance will be created. Does it make sense?



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r853945407


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_NAME;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final Set<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+
+    this.hddsVolumeIDs = new LinkedHashSet<>();
+    if (!b.getFailedVolume()) {
+      LOG.info("Creating DbVolume: {} of storage type : {} capacity : {}",
+          getStorageDir(), b.getStorageType(), getVolumeInfo().getCapacity());
+      initialize();
+    }
+  }
+
+  @Override
+  protected void initialize() throws IOException {
+    super.initialize();
+    scanForHddsVolumeIDs();
+  }
+
+  @Override
+  public void failVolume() {
+    super.failVolume();
+    closeAllDbStore();
+  }
+
+  @Override
+  public void shutdown() {
+    super.shutdown();
+    closeAllDbStore();
+  }
+
+  public void addHddsVolumeID(String id) {
+    hddsVolumeIDs.add(id);
+  }
+
+  public Set<String> getHddsVolumeIDs() {
+    return this.hddsVolumeIDs;
+  }
+
+  /**
+   * Builder class for DbVolume.
+   */
+  public static class Builder extends StorageVolume.Builder<Builder> {
+
+    public Builder(String volumeRootStr) {
+      super(volumeRootStr, DB_VOLUME_DIR);
+    }
+
+    @Override
+    public Builder getThis() {
+      return this;
+    }
+
+    public DbVolume build() throws IOException {
+      return new DbVolume(this);
+    }
+  }
+
+  private void scanForHddsVolumeIDs() throws IOException {
+    // Not formatted yet
+    if (getClusterID() == null) {

Review Comment:
   Will it be better if we detect if state == NORMAL , then continue the process, otherwise return? 
   
   Checking state is more straightforward. 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r853941497


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_NAME;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final Set<String> hddsVolumeIDs;

Review Comment:
   Can we change this Set to a Map, hold both the storageID and db path, so that we can use the db path directly in #closeAllDbStore?



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r854157280


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_NAME;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final Set<String> hddsVolumeIDs;

Review Comment:
   That's a good idea, then we can prevent create some temp file objects.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r854178313


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_NAME;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final Set<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+
+    this.hddsVolumeIDs = new LinkedHashSet<>();
+    if (!b.getFailedVolume()) {
+      LOG.info("Creating DbVolume: {} of storage type : {} capacity : {}",
+          getStorageDir(), b.getStorageType(), getVolumeInfo().getCapacity());
+      initialize();
+    }
+  }
+
+  @Override
+  protected void initialize() throws IOException {
+    super.initialize();
+    scanForHddsVolumeIDs();
+  }
+
+  @Override
+  public void failVolume() {
+    super.failVolume();
+    closeAllDbStore();
+  }
+
+  @Override
+  public void shutdown() {
+    super.shutdown();
+    closeAllDbStore();
+  }
+
+  public void addHddsVolumeID(String id) {
+    hddsVolumeIDs.add(id);
+  }
+
+  public Set<String> getHddsVolumeIDs() {
+    return this.hddsVolumeIDs;
+  }
+
+  /**
+   * Builder class for DbVolume.
+   */
+  public static class Builder extends StorageVolume.Builder<Builder> {
+
+    public Builder(String volumeRootStr) {
+      super(volumeRootStr, DB_VOLUME_DIR);
+    }
+
+    @Override
+    public Builder getThis() {
+      return this;
+    }
+
+    public DbVolume build() throws IOException {
+      return new DbVolume(this);
+    }
+  }
+
+  private void scanForHddsVolumeIDs() throws IOException {
+    // Not formatted yet
+    if (getClusterID() == null) {

Review Comment:
   Well, by reading the routine `initialize` again, I'm sure that check state == NORMAL is correct, because `initialize` will be called in `format` for a second time, then we will have our clusterID.
   But is seems not so straightforward, but a bit complex to understand ?
   Could I keep it to check the clusterID == null here?
   



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848134204


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.ozone.container.common.volume;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  private String clusterID;
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   */
+  private final List<String> hddsVolumeIDs;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+    this.clusterID = b.getClusterID();
+    this.hddsVolumeIDs = new ArrayList<>();
+    if (!b.getFailedVolume()) {
+      initialize();
+    }
+  }
+
+  public boolean format(String cid) {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting db volume");
+    this.clusterID = cid;
+
+    // create clusterID dir /ssd1/db/<CID-clusterID>
+    File volumeRootDir = getStorageDir();
+    File clusterIdDir = new File(volumeRootDir, clusterID);
+    if (!clusterIdDir.mkdirs() && !clusterIdDir.exists()) {
+      LOG.error("Unable to create ID directory {} for db volume {}",
+          clusterIdDir, volumeRootDir);
+      return false;
+    }
+    return true;
+  }
+
+  public boolean initialize() {
+    // This should be on a test path, normally we should get
+    // the clusterID from SCM, and it should not be available
+    // while restarting.
+    if (clusterID != null) {
+      return format(clusterID);
+    }
+
+    if (!getStorageDir().exists()) {
+      // Not formatted yet
+      return true;
+    }
+
+    File[] storageDirs = getStorageDir().listFiles(File::isDirectory);
+    if (storageDirs == null) {
+      LOG.error("IO error for the db volume {}, skipped loading",
+          getStorageDir());
+      return false;

Review Comment:
   Please throw out Exception in all false cases.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r850322484


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   For 2, failed volume check doesn't belong here, it is an async check and DbVolumes are handled as other volumes, please check `StorageVolumeChecker.java`.  And I still want a clear choice between (A)"throw & out" (B)"continue"?
   
   For 3.1, let me explain `hddsFiles.length == 2`:  one file is for VERSION; the other file is either clusterIDDir or scmIDDir, SCM HA checks whether the Dir is (A)clusterIDDir or (B)scmIDDir. If A, do nothing because it means SCM HA is finalized; If B, upgrade this volume for SCM HA by linking. But no matter A or B, we have an IDDir which implies that this HddsVolume is formatted on the first DN register, and we won't do db format again for it neither on DN register nor DN restart. Please check the code inside this function for the words above `VersionedDatanodeFeatures.ScmHA.upgradeVolumeIfNeeded`.
   So here I mean I don't actually depends on SCM HA to check whether we have to format db instance or not and we only format db instance for HddsVolume when `HddsFiles.length == 1`, clear?
   
   For 3.2, after 3.1, I think you should know about the meaning of `hddsFiles.length`. And let me say more about when we have the chance to create a db instance:
   - Along with HddsVolume format on DN first register, so only when we have`hddsFiles.length == 1` which means that we only got a VERSION file on the HddsVolume, so we have to create the clusterIDDir/scmIDDir only on the first DN register.
   - Along with load on DN restart, I think this is where you actually suspect that we may have duplicate db instance creation. But I should say that, there is a check `!storageIdDir.exists()` at line 350 of file `HddsVolumeUtil.java` that checks if the subdirectry with `StorageID` exists, we only load the db instance when this exists. And for now I think I should have a stricter check for the `container.db` directory exactly, then we prevent from creating db instances on all pathes. Make sense?



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848219703


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {

Review Comment:
   Will check volume.getDbVolume() not null be more straight? 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849233705


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   For the HddsVolume -> Instance relation, it is mapped each time on DN startup, we scan each `DbVolume` trying to find the `StorageID` of each `HddsVolume` under it and then we have the map rebuilt, so no need to persist in VERSION file.
   The container replication logic will be introduced in later patches, and containers can be replicated as they are currently. If you are providing an extra command util, that's good.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848235563


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;
+      }
+
+      File storageIdDir = new File(clusterIdDir, volume.getStorageID());
+      if (!storageIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "storageID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              storageIdDir.getAbsolutePath());
+        }
+        continue;
+      }
+      volume.setDbParentDir(storageIdDir);
+
+      String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+          .getAbsolutePath();
+      try {
+        initPerDiskDBStore(containerDBPath, conf);
+      } catch (IOException e) {
+        if (logger != null) {
+          logger.error("Can't load db instance under path {} for volume {}",
+              containerDBPath, volume.getStorageDir().getAbsolutePath());

Review Comment:
   Throw exception here.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849021981


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   For storage, data integrity and security is very important.  So we better be conservative and fail the code fast if something happens which requires admin's notice and interference. So please do throw out the exception here and other places. 
   
   Besides, default value of failedDbVolumeTolerated is -1, which is too optimistic for a storage system. We should change the default value to 0 for all failedVolumeTolerated. But we can do it in a follow up PR.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848279292


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   Here in this function, we are iterating all of the db instances for all `HddsVolume`s, so we don't just throw and bailout.
   We should go on trying to load as many db instances as possible.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,

Review Comment:
   OK



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849021981


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   For storage, data integrity and security is very important.  So we better be conservative and fail the code fast if something happens which requires admin's notice and interference. So please throw out the exception here. 



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848229559


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,

Review Comment:
   formatDbStoreForHddsVolume -> createDbStoreForHddsVolume



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] guihecheng commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
guihecheng commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r848281254


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;
+      }
+
+      File storageIdDir = new File(clusterIdDir, volume.getStorageID());
+      if (!storageIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "storageID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              storageIdDir.getAbsolutePath());
+        }
+        continue;
+      }
+      volume.setDbParentDir(storageIdDir);

Review Comment:
   OK, makes sense.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r847988401


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -144,6 +144,20 @@
       tagged explicitly.
     </description>
   </property>
+  <property>
+    <name>hdds.datanode.container.db.dir</name>
+    <value/>
+    <tag>OZONE, CONTAINER, STORAGE, MANAGEMENT</tag>
+    <description>Determines where the per-disk rocksdb instances will be
+      stored. Defaults to empty if not specified, then rocksdb instances

Review Comment:
   Kindly add the statement "This setting is optional. " here.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r847989229


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -144,6 +144,20 @@
       tagged explicitly.
     </description>
   </property>
+  <property>
+    <name>hdds.datanode.container.db.dir</name>
+    <value/>
+    <tag>OZONE, CONTAINER, STORAGE, MANAGEMENT</tag>
+    <description>Determines where the per-disk rocksdb instances will be
+      stored. Defaults to empty if not specified, then rocksdb instances

Review Comment:
   Defaults to empty if not specified  -> If unspecified



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r849190788


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -234,4 +257,154 @@ public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
       return true;
     }
   }
+
+  /**
+   * Randomly pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param hddsVolume
+   * @param dbVolumeSet
+   * @param clusterID
+   * @param conf
+   */
+  public static void formatDbStoreForHddsVolume(HddsVolume hddsVolume,
+      MutableVolumeSet dbVolumeSet, String clusterID,
+      ConfigurationSource conf) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File dbParentDir;
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      dbParentDir = new File(hddsVolume.getStorageDir(), clusterID);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      dbParentDir = new File(chosenDbVolume.getStorageDir(), clusterID);
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(dbParentDir, hddsVolume.getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + dbParentDir.getAbsolutePath() + " for volume "
+          + hddsVolume.getStorageID());
+    }
+
+    // Init the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, conf);
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + hddsVolume.getStorageID());
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    hddsVolume.setDbVolume(chosenDbVolume);
+    hddsVolume.setDbParentDir(storageIdDir);
+  }
+
+  /**
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
+   * @param conf
+   * @param logger
+   */
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, ConfigurationSource conf, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
+
+    List<HddsVolume> hddsVolumeList = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+
+    for (HddsVolume volume : hddsVolumeList) {
+      String clusterID = volume.getClusterID();
+
+      // DN startup for the first time, not registered yet,
+      // so the DbVolume is not formatted.
+      if (clusterID == null) {
+        continue;
+      }
+
+      File clusterIdDir = new File(volume.getDbVolume() == null ?
+          volume.getStorageDir() : volume.getDbVolume().getStorageDir(),
+          clusterID);
+
+      if (!clusterIdDir.exists()) {
+        if (logger != null) {
+          logger.error("HddsVolume {} db instance not formatted, " +
+                  "clusterID {} directory not found",
+              volume.getStorageDir().getAbsolutePath(),
+              clusterIdDir.getAbsolutePath());
+        }
+        continue;

Review Comment:
   It depends on how many disk failure is tolerated.  For example, in HDFS,  default disk failure tolerated is 0, which means as long as one disk fails, DN cannot start up.  Admin may change the configuration based on his/her estimation of disk failure rate.  That's acceptable and the risk is on Admin.  So "at least one disk volume" is not acceptable, the risk is too high. 
   
   Think about this case, all HDDS volumes function well while one of the two configured dbVolumes is down at DN startup.  How will the impacted HDDS volume behave? Create a new RocksDB instance on the remaining dbVolume and then go on to provide service?
    I think we should persist this HDDS volume to RocksDB instance relation in HDDS version file.   And we also need a replicate container meta data command to recovery the rocksdb content for a dedicated container.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3292: HDDS-6541. [Merge rocksdb in datanode] Per-disk DB location management.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on code in PR #3292:
URL: https://github.com/apache/ozone/pull/3292#discussion_r853913394


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java:
##########
@@ -133,6 +138,16 @@ public OzoneContainer(
     volumeSet.setFailedVolumeListener(this::handleVolumeFailures);
     metaVolumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(), conf,
         context, VolumeType.META_VOLUME, volumeChecker);
+    if (VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion()
+        .equals(OzoneConsts.SCHEMA_V3)) {
+      dbVolumeSet = HddsServerUtil.getDatanodeDbDirs(conf).isEmpty() ? null :

Review Comment:
   I created HDDS-6616 to track this new db scanner requirement.



-- 
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@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org