Browse Source

HDFS-12454. Ozone : the sample ozone-site.xml in OzoneGettingStarted does not work. Contributed by Chen Liang.

Anu Engineer 7 years ago
parent
commit
a1a3ba6529
21 changed files with 87 additions and 53 deletions
  1. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
  2. 2 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/ScmConfigKeys.java
  3. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java
  4. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
  5. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
  6. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
  7. 26 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/ozone-default.xml
  9. 27 22
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/OzoneGettingStarted.md.vm
  10. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
  11. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
  13. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMSQLCli.java
  15. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/block/TestBlockManager.java
  17. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/block/TestDeletedBlockLog.java
  18. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerMapping.java
  19. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/node/TestContainerPlacement.java
  20. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/node/TestNodeManager.java
  21. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/node/TestSCMNodePoolManager.java

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java

@@ -77,8 +77,8 @@ public final class OzoneConfigKeys {
       "ozone.trace.enabled";
       "ozone.trace.enabled";
   public static final boolean OZONE_TRACE_ENABLED_DEFAULT = false;
   public static final boolean OZONE_TRACE_ENABLED_DEFAULT = false;
 
 
-  public static final String OZONE_CONTAINER_METADATA_DIRS =
-      "ozone.container.metadata.dirs";
+  public static final String OZONE_METADATA_DIRS =
+      "ozone.metadata.dirs";
 
 
   public static final String OZONE_METADATA_STORE_IMPL =
   public static final String OZONE_METADATA_STORE_IMPL =
       "ozone.metastore.impl";
       "ozone.metastore.impl";

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/ScmConfigKeys.java

@@ -171,6 +171,8 @@ public final class ScmConfigKeys {
   // if this value is not set then container startup will fail.
   // if this value is not set then container startup will fail.
   public static final String OZONE_SCM_DATANODE_ID = "ozone.scm.datanode.id";
   public static final String OZONE_SCM_DATANODE_ID = "ozone.scm.datanode.id";
 
 
+  public static final String OZONE_SCM_DATANODE_ID_PATH_DEFAULT = "datanode.id";
+
   public static final String OZONE_SCM_DB_CACHE_SIZE_MB =
   public static final String OZONE_SCM_DB_CACHE_SIZE_MB =
       "ozone.scm.db.cache.size.mb";
       "ozone.scm.db.cache.size.mb";
   public static final int OZONE_SCM_DB_CACHE_SIZE_DEFAULT = 128;
   public static final int OZONE_SCM_DB_CACHE_SIZE_DEFAULT = 128;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachin
 import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
 import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.container.common.states.DatanodeState;
 import org.apache.hadoop.ozone.container.common.states.DatanodeState;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.scm.ScmConfigKeys;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
@@ -104,7 +105,7 @@ public class InitDatanodeState implements DatanodeState,
    * and persist the ID to a local file.
    * and persist the ID to a local file.
    */
    */
   private void persistContainerDatanodeID() throws IOException {
   private void persistContainerDatanodeID() throws IOException {
-    String dataNodeIDPath = conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID);
+    String dataNodeIDPath = OzoneUtils.getDatanodeIDPath(conf);
     if (Strings.isNullOrEmpty(dataNodeIDPath)) {
     if (Strings.isNullOrEmpty(dataNodeIDPath)) {
       LOG.error("A valid file path is needed for config setting {}",
       LOG.error("A valid file path is needed for config setting {}",
           ScmConfigKeys.OZONE_SCM_DATANODE_ID);
           ScmConfigKeys.OZONE_SCM_DATANODE_ID);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.ozone.container.common.states.endpoint.HeartbeatEndpoin
 import org.apache.hadoop.ozone.container.common.states.endpoint.RegisterEndpointTask;
 import org.apache.hadoop.ozone.container.common.states.endpoint.RegisterEndpointTask;
 import org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask;
 import org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask;
 import org.apache.hadoop.ozone.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
@@ -112,7 +113,7 @@ public class RunningDatanodeState implements DatanodeState {
    */
    */
   private StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto
   private StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto
       getContainerNodeID() {
       getContainerNodeID() {
-    String dataNodeIDPath = conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID);
+    String dataNodeIDPath = OzoneUtils.getDatanodeIDPath(conf);
     if (dataNodeIDPath == null || dataNodeIDPath.isEmpty()) {
     if (dataNodeIDPath == null || dataNodeIDPath.isEmpty()) {
       LOG.error("A valid file path is needed for config setting {}",
       LOG.error("A valid file path is needed for config setting {}",
           ScmConfigKeys.OZONE_SCM_DATANODE_ID);
           ScmConfigKeys.OZONE_SCM_DATANODE_ID);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java

@@ -97,8 +97,8 @@ public final class XceiverServerRatis implements XceiverServerSpi {
 
 
     if (Strings.isNullOrEmpty(storageDir)) {
     if (Strings.isNullOrEmpty(storageDir)) {
       storageDir = ozoneConf.get(OzoneConfigKeys
       storageDir = ozoneConf.get(OzoneConfigKeys
-          .OZONE_CONTAINER_METADATA_DIRS);
-      Preconditions.checkNotNull(storageDir, "ozone.container.metadata.dirs " +
+          .OZONE_METADATA_DIRS);
+      Preconditions.checkNotNull(storageDir, "ozone.metadata.dirs " +
           "cannot be null, Please check your configs.");
           "cannot be null, Please check your configs.");
       storageDir = storageDir.concat(ratisDir);
       storageDir = storageDir.concat(ratisDir);
       LOG.warn("Storage directory for Ratis is not configured. Mapping Ratis " +
       LOG.warn("Storage directory for Ratis is not configured. Mapping Ratis " +

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java

@@ -85,7 +85,7 @@ public class OzoneContainer {
     this.ozoneConfig = ozoneConfig;
     this.ozoneConfig = ozoneConfig;
     List<StorageLocation> locations = new LinkedList<>();
     List<StorageLocation> locations = new LinkedList<>();
     String[] paths = ozoneConfig.getStrings(
     String[] paths = ozoneConfig.getStrings(
-        OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS);
+        OzoneConfigKeys.OZONE_METADATA_DIRS);
     if (paths != null && paths.length > 0) {
     if (paths != null && paths.length > 0) {
       for (String p : paths) {
       for (String p : paths) {
         locations.add(StorageLocation.parse(
         locations.add(StorageLocation.parse(

+ 26 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.utils;
 package org.apache.hadoop.ozone.web.utils;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -28,6 +29,7 @@ import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.client.rest.headers.Header;
+import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 
 
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.HttpHeaders;
@@ -38,6 +40,7 @@ import java.io.File;
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
 import java.nio.charset.Charset;
 import java.nio.charset.Charset;
+import java.nio.file.Paths;
 import java.text.ParseException;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Date;
@@ -316,7 +319,7 @@ public final class OzoneUtils {
    */
    */
   public static File getScmMetadirPath(Configuration conf) {
   public static File getScmMetadirPath(Configuration conf) {
     String metaDirPath = conf.getTrimmed(OzoneConfigKeys
     String metaDirPath = conf.getTrimmed(OzoneConfigKeys
-        .OZONE_CONTAINER_METADATA_DIRS);
+        .OZONE_METADATA_DIRS);
     Preconditions.checkNotNull(metaDirPath);
     Preconditions.checkNotNull(metaDirPath);
     File dirPath = new File(metaDirPath);
     File dirPath = new File(metaDirPath);
     if (!dirPath.exists() && !dirPath.mkdirs()) {
     if (!dirPath.exists() && !dirPath.mkdirs()) {
@@ -326,6 +329,28 @@ public final class OzoneUtils {
     return dirPath;
     return dirPath;
   }
   }
 
 
+  /**
+   * Get the path for datanode id file.
+   *
+   * @param conf - Configuration
+   * @return the path of datanode id as string
+   */
+  public static String getDatanodeIDPath(Configuration conf) {
+    String dataNodeIDPath = conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID);
+    if (Strings.isNullOrEmpty(dataNodeIDPath)) {
+      String metaPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
+      if (Strings.isNullOrEmpty(metaPath)) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode id path");
+      }
+      dataNodeIDPath = Paths.get(metaPath,
+          ScmConfigKeys.OZONE_SCM_DATANODE_ID_PATH_DEFAULT).toString();
+    }
+    return dataNodeIDPath;
+  }
+
   /**
   /**
    * Convert time in millisecond to a human readable format required in ozone.
    * Convert time in millisecond to a human readable format required in ozone.
    * @return a human readable string for the input time
    * @return a human readable string for the input time

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/ozone-default.xml

@@ -59,7 +59,7 @@
   </property>
   </property>
 
 
   <property>
   <property>
-    <name>ozone.container.metadata.dirs</name>
+    <name>ozone.metadata.dirs</name>
     <value></value>
     <value></value>
     <description>
     <description>
       Ozone metadata dir path.
       Ozone metadata dir path.

+ 27 - 22
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/OzoneGettingStarted.md.vm

@@ -63,17 +63,17 @@ place  and not mingled with HDFS settings.
        <value>True</value>
        <value>True</value>
     </property>
     </property>
 ```
 ```
- *  _*ozone.container.metadata.dirs*_ Ozone is designed with modern hardware
+ *  _*ozone.metadata.dirs*_ Ozone is designed with modern hardware
  in mind. It tries to use SSDs effectively. So users can specify where the
  in mind. It tries to use SSDs effectively. So users can specify where the
- datanode metadata must reside. Usually you pick your fastest disk (SSD if
- you have them on your datanodes). Datanodes will write the container metadata
- to these disks. This is a required setting, if this is missing datanodes will
+ metadata must reside. Usually you pick your fastest disk (SSD if
+ you have them on your nodes). KSM, SCM and datanode will write the metadata
+ to these disks. This is a required setting, if this is missing Ozone will
  fail to come up. Here is an example,
  fail to come up. Here is an example,
 
 
 ```
 ```
    <property>
    <property>
-      <name>ozone.container.metadata.dirs</name>
-      <value>/data/disk1/container/meta</value>
+      <name>ozone.metadata.dirs</name>
+      <value>/data/disk1/meta</value>
    </property>
    </property>
 ```
 ```
 
 
@@ -135,10 +135,11 @@ Here is a quick summary of settings needed by Ozone.
 | Setting                        | Value                        | Comment |
 | Setting                        | Value                        | Comment |
 |--------------------------------|------------------------------|------------------------------------------------------------------|
 |--------------------------------|------------------------------|------------------------------------------------------------------|
 | ozone.enabled                  | True                         | This enables SCM and  containers in HDFS cluster.                |
 | ozone.enabled                  | True                         | This enables SCM and  containers in HDFS cluster.                |
-| ozone.container.metadata.dirs  | file path                    | The container metadata will be stored here in the datanode.      |
+| ozone.metadata.dirs            | file path                    | The metadata will be stored here.                                |
 | ozone.scm.names                | SCM server name              | Hostname:port or or IP:port address of SCM.                      |
 | ozone.scm.names                | SCM server name              | Hostname:port or or IP:port address of SCM.                      |
-| ozone.scm.datanode.id          | file path                    | Data node ID is the location of  datanode's ID file              |
-| ozone.scm.block.client.address | SCM server name              | Used by services like KSM                                        |
+| ozone.scm.block.client.address | SCM server name and port     | Used by services like KSM                                        |
+| ozone.scm.client.address       | SCM server name and port     | Used by client side                                              |
+| ozone.scm.datanode.address     | SCM server name and port     | Used by datanode to talk to SCM                                  |
 | ozone.ksm.address              | KSM server name              | Used by Ozone handler and Ozone file system.                     |
 | ozone.ksm.address              | KSM server name              | Used by Ozone handler and Ozone file system.                     |
 
 
  Here is a working example of`ozone-site.xml`.
  Here is a working example of`ozone-site.xml`.
@@ -153,30 +154,34 @@ Here is a quick summary of settings needed by Ozone.
         </property>
         </property>
 
 
         <property>
         <property>
-          <name>ozone.container.metadata.dirs</name>
-          <value>/data/disk1/scm/meta</value>
+          <name>ozone.metadata.dirs</name>
+          <value>/data/disk1/ozone/meta</value>
         </property>
         </property>
 
 
-
         <property>
         <property>
           <name>ozone.scm.names</name>
           <name>ozone.scm.names</name>
-          <value>scm.hadoop.apache.org</value>
+          <value>127.0.0.1</value>
         </property>
         </property>
 
 
         <property>
         <property>
-          <name>ozone.scm.datanode.id</name>
-          <value>/data/disk1/scm/meta/node/datanode.id</value>
+           <name>ozone.scm.client.address</name>
+           <value>127.0.0.1:9860</value>
         </property>
         </property>
 
 
-        <property>
-          <name>ozone.scm.block.client.address</name>
-          <value>scm.hadoop.apache.org</value>
-        </property>
+         <property>
+           <name>ozone.scm.block.client.address</name>
+           <value>127.0.0.1:9863</value>
+         </property>
+
+         <property>
+           <name>ozone.scm.datanode.address</name>
+           <value>127.0.0.1:9861</value>
+         </property>
 
 
          <property>
          <property>
-            <name>ozone.ksm.address</name>
-            <value>ksm.hadoop.apache.org</value>
-          </property>
+           <name>ozone.ksm.address</name>
+           <value>127.0.0.1:9874</value>
+         </property>
     </configuration>
     </configuration>
 ```
 ```
 
 

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java

@@ -113,9 +113,9 @@ public final class MiniOzoneCluster extends MiniDFSCluster
     setConf(i, dnConf, OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR,
     setConf(i, dnConf, OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR,
         getInstanceStorageDir(i, -1).getCanonicalPath());
         getInstanceStorageDir(i, -1).getCanonicalPath());
     String containerMetaDirs = dnConf.get(
     String containerMetaDirs = dnConf.get(
-        OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS) + "-dn-" + i;
+        OzoneConfigKeys.OZONE_METADATA_DIRS) + "-dn-" + i;
     Path containerMetaDirPath = Paths.get(containerMetaDirs);
     Path containerMetaDirPath = Paths.get(containerMetaDirs);
-    setConf(i, dnConf, OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS,
+    setConf(i, dnConf, OzoneConfigKeys.OZONE_METADATA_DIRS,
         containerMetaDirs);
         containerMetaDirs);
     Path containerRootPath =
     Path containerRootPath =
         containerMetaDirPath.resolve(OzoneConsts.CONTAINER_ROOT_PREFIX);
         containerMetaDirPath.resolve(OzoneConsts.CONTAINER_ROOT_PREFIX);
@@ -476,7 +476,7 @@ public final class MiniOzoneCluster extends MiniDFSCluster
       if (scmMetadataDir.isPresent()) {
       if (scmMetadataDir.isPresent()) {
         // if user specifies a path in the test, it is assumed that user takes
         // if user specifies a path in the test, it is assumed that user takes
         // care of creating and cleaning up that directory after the tests.
         // care of creating and cleaning up that directory after the tests.
-        conf.set(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS,
+        conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
             scmMetadataDir.get());
             scmMetadataDir.get());
         return;
         return;
       }
       }
@@ -487,7 +487,7 @@ public final class MiniOzoneCluster extends MiniDFSCluster
       Files.createDirectories(scmPath);
       Files.createDirectories(scmPath);
       Path containerPath = scmPath.resolve(OzoneConsts.CONTAINER_ROOT_PREFIX);
       Path containerPath = scmPath.resolve(OzoneConsts.CONTAINER_ROOT_PREFIX);
       Files.createDirectories(containerPath);
       Files.createDirectories(containerPath);
-      conf.set(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS, scmPath
+      conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, scmPath
           .toString());
           .toString());
 
 
       // TODO : Fix this, we need a more generic mechanism to map
       // TODO : Fix this, we need a more generic mechanism to map

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java

@@ -44,7 +44,7 @@ import java.util.List;
 
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
 /**
 /**
@@ -62,7 +62,7 @@ public class TestMiniOzoneCluster {
   @BeforeClass
   @BeforeClass
   public static void setup() {
   public static void setup() {
     conf = new OzoneConfiguration();
     conf = new OzoneConfiguration();
-    conf.set(OZONE_CONTAINER_METADATA_DIRS,
+    conf.set(OZONE_METADATA_DIRS,
         TEST_ROOT.toString());
         TEST_ROOT.toString());
     conf.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, true);
     conf.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, true);
     WRITE_TMP.mkdirs();
     WRITE_TMP.mkdirs();
@@ -183,7 +183,7 @@ public class TestMiniOzoneCluster {
     Configuration ozoneConf = SCMTestUtils.getConf();
     Configuration ozoneConf = SCMTestUtils.getConf();
     File testDir = PathUtils.getTestDir(TestOzoneContainer.class);
     File testDir = PathUtils.getTestDir(TestOzoneContainer.class);
     ozoneConf.set(DFS_DATANODE_DATA_DIR_KEY, testDir.getAbsolutePath());
     ozoneConf.set(DFS_DATANODE_DATA_DIR_KEY, testDir.getAbsolutePath());
-    ozoneConf.set(OZONE_CONTAINER_METADATA_DIRS,
+    ozoneConf.set(OZONE_METADATA_DIRS,
         TEST_ROOT.toString());
         TEST_ROOT.toString());
 
 
     // Each instance of SM will create an ozone container
     // Each instance of SM will create an ozone container

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java

@@ -102,7 +102,7 @@ public class TestDatanodeStateMachine {
     }
     }
     conf.set(DFS_DATANODE_DATA_DIR_KEY,
     conf.set(DFS_DATANODE_DATA_DIR_KEY,
         new File(testRoot, "data").getAbsolutePath());
         new File(testRoot, "data").getAbsolutePath());
-    conf.set(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS,
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
         new File(testRoot, "scm").getAbsolutePath());
         new File(testRoot, "scm").getAbsolutePath());
     path = Paths.get(path.toString(),
     path = Paths.get(path.toString(),
         TestDatanodeStateMachine.class.getSimpleName() + ".id").toString();
         TestDatanodeStateMachine.class.getSimpleName() + ".id").toString();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java

@@ -64,7 +64,7 @@ import java.util.UUID;
 
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.ozone.OzoneConfigKeys
 import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_CONTAINER_METADATA_DIRS;
+    .OZONE_METADATA_DIRS;
 import static org.apache.hadoop.ozone.container.common.SCMTestUtils
 import static org.apache.hadoop.ozone.container.common.SCMTestUtils
     .getDatanodeID;
     .getDatanodeID;
 import static org.apache.hadoop.ozone.protocol.proto
 import static org.apache.hadoop.ozone.protocol.proto
@@ -298,7 +298,7 @@ public class TestEndPoint {
       int rpcTimeout) throws Exception {
       int rpcTimeout) throws Exception {
     Configuration conf = SCMTestUtils.getConf();
     Configuration conf = SCMTestUtils.getConf();
     conf.set(DFS_DATANODE_DATA_DIR_KEY, testDir.getAbsolutePath());
     conf.set(DFS_DATANODE_DATA_DIR_KEY, testDir.getAbsolutePath());
-    conf.set(OZONE_CONTAINER_METADATA_DIRS, testDir.getAbsolutePath());
+    conf.set(OZONE_METADATA_DIRS, testDir.getAbsolutePath());
     // Mini Ozone cluster will not come up if the port is not true, since
     // Mini Ozone cluster will not come up if the port is not true, since
     // Ratis will exit if the server port cannot be bound. We can remove this
     // Ratis will exit if the server port cannot be bound. We can remove this
     // hard coding once we fix the Ratis default behaviour.
     // hard coding once we fix the Ratis default behaviour.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMSQLCli.java

@@ -159,7 +159,7 @@ public class TestKSMSQLCli {
   public void testKSMDB() throws Exception {
   public void testKSMDB() throws Exception {
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
 
 
-    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS);
+    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
     String dbPath = dbRootPath + "/" + KSM_DB_NAME;
     String dbPath = dbRootPath + "/" + KSM_DB_NAME;
     String[] args = {"-p", dbPath, "-o", dbOutPath};
     String[] args = {"-p", dbPath, "-o", dbOutPath};
 
 

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java

@@ -171,7 +171,7 @@ public class TestContainerSQLCli {
   @Test
   @Test
   public void testConvertBlockDB() throws Exception {
   public void testConvertBlockDB() throws Exception {
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
-    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS);
+    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
     String dbPath = dbRootPath + "/" + BLOCK_DB;
     String dbPath = dbRootPath + "/" + BLOCK_DB;
     String[] args = {"-p", dbPath, "-o", dbOutPath};
     String[] args = {"-p", dbPath, "-o", dbOutPath};
 
 
@@ -193,7 +193,7 @@ public class TestContainerSQLCli {
   @Test
   @Test
   public void testConvertNodepoolDB() throws Exception {
   public void testConvertNodepoolDB() throws Exception {
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
-    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS);
+    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
     String dbPath = dbRootPath + "/" + NODEPOOL_DB;
     String dbPath = dbRootPath + "/" + NODEPOOL_DB;
     String[] args = {"-p", dbPath, "-o", dbOutPath};
     String[] args = {"-p", dbPath, "-o", dbOutPath};
 
 
@@ -220,7 +220,7 @@ public class TestContainerSQLCli {
   @Test
   @Test
   public void testConvertOpenContainerDB() throws Exception {
   public void testConvertOpenContainerDB() throws Exception {
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
-    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS);
+    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
     String dbPath = dbRootPath + "/" + OPEN_CONTAINERS_DB;
     String dbPath = dbRootPath + "/" + OPEN_CONTAINERS_DB;
     String[] args = {"-p", dbPath, "-o", dbOutPath};
     String[] args = {"-p", dbPath, "-o", dbOutPath};
 
 
@@ -254,7 +254,7 @@ public class TestContainerSQLCli {
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
     // TODO : the following will fail due to empty Datanode list, need to fix.
     // TODO : the following will fail due to empty Datanode list, need to fix.
     //String dnUUID = cluster.getDataNodes().get(0).getDatanodeUuid();
     //String dnUUID = cluster.getDataNodes().get(0).getDatanodeUuid();
-    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS);
+    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
     String dbPath = dbRootPath + "/" + CONTAINER_DB;
     String dbPath = dbRootPath + "/" + CONTAINER_DB;
     String[] args = {"-p", dbPath, "-o", dbOutPath};
     String[] args = {"-p", dbPath, "-o", dbOutPath};
     Connection conn;
     Connection conn;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/block/TestBlockManager.java

@@ -63,7 +63,7 @@ public class TestBlockManager {
     String path = GenericTestUtils
     String path = GenericTestUtils
         .getTempPath(TestBlockManager.class.getSimpleName());
         .getTempPath(TestBlockManager.class.getSimpleName());
 
 
-    conf.set(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS, path);
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, path);
     testDir = Paths.get(path).toFile();
     testDir = Paths.get(path).toFile();
     boolean folderExisted = testDir.exists() || testDir.mkdirs();
     boolean folderExisted = testDir.exists() || testDir.mkdirs();
     if (!folderExisted) {
     if (!folderExisted) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/block/TestDeletedBlockLog.java

@@ -40,7 +40,7 @@ import java.util.Random;
 import java.util.UUID;
 import java.util.UUID;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
 
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
 import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_BLOCK_DELETION_MAX_RETRY;
 import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_BLOCK_DELETION_MAX_RETRY;
 
 
 /**
 /**
@@ -58,7 +58,7 @@ public class TestDeletedBlockLog {
         TestDeletedBlockLog.class.getSimpleName());
         TestDeletedBlockLog.class.getSimpleName());
     conf = new OzoneConfiguration();
     conf = new OzoneConfiguration();
     conf.setInt(OZONE_SCM_BLOCK_DELETION_MAX_RETRY, 20);
     conf.setInt(OZONE_SCM_BLOCK_DELETION_MAX_RETRY, 20);
-    conf.set(OZONE_CONTAINER_METADATA_DIRS, testDir.getAbsolutePath());
+    conf.set(OZONE_METADATA_DIRS, testDir.getAbsolutePath());
     deletedBlockLog = new DeletedBlockLogImpl(conf);
     deletedBlockLog = new DeletedBlockLogImpl(conf);
   }
   }
 
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerMapping.java

@@ -55,7 +55,7 @@ public class TestContainerMapping {
 
 
     testDir = GenericTestUtils
     testDir = GenericTestUtils
         .getTestDir(TestContainerMapping.class.getSimpleName());
         .getTestDir(TestContainerMapping.class.getSimpleName());
-    conf.set(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS,
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
         testDir.getAbsolutePath());
         testDir.getAbsolutePath());
     boolean folderExisted = testDir.exists() || testDir.mkdirs();
     boolean folderExisted = testDir.exists() || testDir.mkdirs();
     if (!folderExisted) {
     if (!folderExisted) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/node/TestContainerPlacement.java

@@ -115,7 +115,7 @@ public class TestContainerPlacement {
 
 
     final File testDir = PathUtils.getTestDir(
     final File testDir = PathUtils.getTestDir(
         TestContainerPlacement.class);
         TestContainerPlacement.class);
-    conf.set(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS,
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
         testDir.getAbsolutePath());
         testDir.getAbsolutePath());
     conf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
     conf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
         SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);
         SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/node/TestNodeManager.java

@@ -102,7 +102,7 @@ public class TestNodeManager {
    */
    */
   OzoneConfiguration getConf() {
   OzoneConfiguration getConf() {
     OzoneConfiguration conf = new OzoneConfiguration();
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS,
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
         testDir.getAbsolutePath());
         testDir.getAbsolutePath());
     conf.setLong(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS, 100);
     conf.setLong(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS, 100);
     return conf;
     return conf;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/node/TestSCMNodePoolManager.java

@@ -57,7 +57,7 @@ public class TestSCMNodePoolManager {
 
 
   SCMNodePoolManager createNodePoolManager(OzoneConfiguration conf)
   SCMNodePoolManager createNodePoolManager(OzoneConfiguration conf)
       throws IOException {
       throws IOException {
-    conf.set(OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS,
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
         testDir.getAbsolutePath());
         testDir.getAbsolutePath());
     conf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
     conf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
         SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);
         SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);