Prechádzať zdrojové kódy

HDFS-11071. Ozone: SCM: Move SCM config keys to ScmConfig. Contributed by Weiwei Yang.

Xiaoyu Yao 8 rokov pred
rodič
commit
8eca9824cf

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

@@ -37,4 +37,90 @@ public final class ScmConfigKeys {
 
   // TODO : this is copied from OzoneConsts, may need to move to a better place
   public static final int CHUNK_SIZE = 1 * 1024 * 1024; // 1 MB
+
+  public static final int OZONE_SCM_CLIENT_PORT_DEFAULT = 9860;
+  public static final int OZONE_SCM_DATANODE_PORT_DEFAULT = 9861;
+
+  public static final String OZONE_SCM_CLIENT_ADDRESS_KEY =
+      "ozone.scm.client.address";
+  public static final String OZONE_SCM_CLIENT_BIND_HOST_KEY =
+      "ozone.scm.client.bind.host";
+  public static final String OZONE_SCM_CLIENT_BIND_HOST_DEFAULT =
+      "0.0.0.0";
+
+  public static final String OZONE_SCM_DATANODE_ADDRESS_KEY =
+      "ozone.scm.datanode.address";
+  public static final String OZONE_SCM_DATANODE_BIND_HOST_KEY =
+      "ozone.scm.datanode.bind.host";
+  public static final String OZONE_SCM_DATANODE_BIND_HOST_DEFAULT =
+      "0.0.0.0";
+
+  public static final String OZONE_SCM_HANDLER_COUNT_KEY =
+      "ozone.scm.handler.count.key";
+  public static final int OZONE_SCM_HANDLER_COUNT_DEFAULT = 10;
+
+  public static final String OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS =
+      "ozone.scm.heartbeat.interval.seconds";
+  public static final int OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT =
+      30;
+
+  public static final String OZONE_SCM_DEADNODE_INTERVAL_MS =
+      "ozone.scm.dead.node.interval.ms";
+  public static final long OZONE_SCM_DEADNODE_INTERVAL_DEFAULT =
+      OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT * 1000L * 20L;
+
+  public static final String OZONE_SCM_MAX_HB_COUNT_TO_PROCESS =
+      "ozone.scm.max.hb.count.to.process";
+  public static final int OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT = 5000;
+
+  public static final String OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS =
+      "ozone.scm.heartbeat.thread.interval.ms";
+  public static final long OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS_DEFAULT =
+      3000;
+
+  public static final String OZONE_SCM_STALENODE_INTERVAL_MS =
+      "ozone.scm.stale.node.interval.ms";
+  public static final long OZONE_SCM_STALENODE_INTERVAL_DEFAULT =
+      OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT * 1000L * 3L;
+
+  public static final String OZONE_SCM_HEARTBEAT_RPC_TIMEOUT =
+      "ozone.scm.heartbeat.rpc-timeout";
+  public static final long OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT =
+      100;
+
+  /**
+   * Defines how frequently we will log the missing of heartbeat to a specific
+   * SCM. In the default case we will write a warning message for each 10
+   * sequential heart beats that we miss to a specific SCM. This is to avoid
+   * overrunning the log with lots of HB missed Log statements.
+   */
+  public static final String OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT =
+      "ozone.scm.heartbeat.log.warn.interval.count";
+  public static final int OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT =
+      10;
+
+  // ozone.scm.names key is a set of DNS | DNS:PORT | IP Address | IP:PORT.
+  // Written as a comma separated string. e.g. scm1, scm2:8020, 7.7.7.7:7777
+  //
+  // If this key is not specified datanodes will not be able to find
+  // SCM. The SCM membership can be dynamic, so this key should contain
+  // all possible SCM names. Once the SCM leader is discovered datanodes will
+  // get the right list of SCMs to heartbeat to from the leader.
+  // While it is good for the datanodes to know the names of all SCM nodes,
+  // it is sufficient to actually know the name of on working SCM. That SCM
+  // will be able to return the information about other SCMs that are part of
+  // the SCM replicated Log.
+  //
+  //In case of a membership change, any one of the SCM machines will be
+  // able to send back a new list to the datanodes.
+  public static final String OZONE_SCM_NAMES = "ozone.scm.names";
+
+  public static final int OZONE_SCM_DEFAULT_PORT = 9862;
+  // File Name and path where datanode ID is to written to.
+  // 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_DB_CACHE_SIZE_MB =
+      "ozone.scm.db.cache.size.mb";
+  public static final int OZONE_SCM_DB_CACHE_SIZE_DEFAULT = 128;
 }

+ 39 - 35
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientUtils.java

@@ -35,28 +35,27 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys.*;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_DEADNODE_INTERVAL_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_DEADNODE_INTERVAL_MS;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS;
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS;
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_HEARTBEAT_RPC_TIMEOUT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_STALENODE_INTERVAL_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_STALENODE_INTERVAL_MS;
 
 /**
@@ -96,15 +95,15 @@ public final class OzoneClientUtils {
     Collection<InetSocketAddress> addresses =
         new HashSet<InetSocketAddress>();
     Collection<String> names =
-        conf.getTrimmedStringCollection(OzoneConfigKeys.OZONE_SCM_NAMES);
+        conf.getTrimmedStringCollection(ScmConfigKeys.OZONE_SCM_NAMES);
     if (names == null || names.isEmpty()) {
-      throw new IllegalArgumentException(OzoneConfigKeys.OZONE_SCM_NAMES
+      throw new IllegalArgumentException(ScmConfigKeys.OZONE_SCM_NAMES
           + " need to be a set of valid DNS names or IP addresses."
           + " Null or empty address list found.");
     }
 
     final com.google.common.base.Optional<Integer>
-        defaultPort =  com.google.common.base.Optional.of(OzoneConfigKeys
+        defaultPort =  com.google.common.base.Optional.of(ScmConfigKeys
         .OZONE_SCM_DEFAULT_PORT);
     for (String address : names) {
       com.google.common.base.Optional<String> hostname =
@@ -131,20 +130,21 @@ public final class OzoneClientUtils {
    */
   public static InetSocketAddress getScmAddressForClients(Configuration conf) {
     final Optional<String> host = getHostNameFromConfigKeys(conf,
-        OZONE_SCM_CLIENT_ADDRESS_KEY);
+        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
 
     if (!host.isPresent()) {
-      throw new IllegalArgumentException(OZONE_SCM_CLIENT_ADDRESS_KEY +
+      throw new IllegalArgumentException(
+          ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY +
           " must be defined. See" +
           " https://wiki.apache.org/hadoop/Ozone#Configuration for details" +
           " on configuring Ozone.");
     }
 
     final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        OZONE_SCM_CLIENT_ADDRESS_KEY);
+        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
 
     return NetUtils.createSocketAddr(host.get() + ":" +
-        port.or(OZONE_SCM_CLIENT_PORT_DEFAULT));
+        port.or(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
   }
 
   /**
@@ -162,10 +162,12 @@ public final class OzoneClientUtils {
     // - OZONE_SCM_CLIENT_ADDRESS_KEY
     //
     final Optional<String> host = getHostNameFromConfigKeys(conf,
-        OZONE_SCM_DATANODE_ADDRESS_KEY, OZONE_SCM_CLIENT_ADDRESS_KEY);
+        ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY,
+        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
 
     if (!host.isPresent()) {
-      throw new IllegalArgumentException(OZONE_SCM_CLIENT_ADDRESS_KEY +
+      throw new IllegalArgumentException(
+          ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY +
           " must be defined. See" +
           " https://wiki.apache.org/hadoop/Ozone#Configuration for details" +
           " on configuring Ozone.");
@@ -173,10 +175,10 @@ public final class OzoneClientUtils {
 
     // If no port number is specified then we'll just try the defaultBindPort.
     final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        OZONE_SCM_DATANODE_ADDRESS_KEY);
+        ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY);
 
     InetSocketAddress addr = NetUtils.createSocketAddr(host.get() + ":" +
-        port.or(OZONE_SCM_DATANODE_PORT_DEFAULT));
+        port.or(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
 
     return addr;
   }
@@ -191,14 +193,14 @@ public final class OzoneClientUtils {
   public static InetSocketAddress getScmClientBindAddress(
       Configuration conf) {
     final Optional<String> host = getHostNameFromConfigKeys(conf,
-        OZONE_SCM_CLIENT_BIND_HOST_KEY);
+        ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_KEY);
 
     final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        OZONE_SCM_CLIENT_ADDRESS_KEY);
+        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
 
     return NetUtils.createSocketAddr(
-        host.or(OZONE_SCM_CLIENT_BIND_HOST_DEFAULT) + ":" +
-            port.or(OZONE_SCM_CLIENT_PORT_DEFAULT));
+        host.or(ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_DEFAULT) + ":" +
+            port.or(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
   }
 
   /**
@@ -211,15 +213,15 @@ public final class OzoneClientUtils {
   public static InetSocketAddress getScmDataNodeBindAddress(
       Configuration conf) {
     final Optional<String> host = getHostNameFromConfigKeys(conf,
-        OZONE_SCM_DATANODE_BIND_HOST_KEY);
+        ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_KEY);
 
     // If no port number is specified then we'll just try the defaultBindPort.
     final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        OZONE_SCM_DATANODE_ADDRESS_KEY);
+        ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY);
 
     return NetUtils.createSocketAddr(
-        host.or(OZONE_SCM_DATANODE_BIND_HOST_DEFAULT) + ":" +
-            port.or(OZONE_SCM_DATANODE_PORT_DEFAULT));
+        host.or(ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_DEFAULT) + ":" +
+            port.or(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
   }
 
   /**
@@ -351,7 +353,7 @@ public final class OzoneClientUtils {
    */
   public static long getScmheartbeatCheckerInterval(Configuration conf) {
     return conf.getLong(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS,
-        OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS_DEFAULT);
+        ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS_DEFAULT);
   }
 
   /**
@@ -362,8 +364,10 @@ public final class OzoneClientUtils {
    * @return - HB interval in seconds.
    */
   public static long getScmHeartbeatInterval(Configuration conf) {
-    return conf.getTimeDuration(OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS,
-        OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT, TimeUnit.SECONDS);
+    return conf.getTimeDuration(
+        OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS,
+        ScmConfigKeys.OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT,
+        TimeUnit.SECONDS);
   }
 
   /**
@@ -444,8 +448,8 @@ public final class OzoneClientUtils {
    * @return - int -- Number of HBs to process
    */
   public static int getMaxHBToProcessPerLoop(Configuration conf) {
-    return conf.getInt(OZONE_SCM_MAX_HB_COUNT_TO_PROCESS,
-        OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT);
+    return conf.getInt(ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS,
+        ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT);
   }
 
   /**

+ 0 - 88
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java

@@ -49,98 +49,10 @@ public final class OzoneConfigKeys {
   public static final String OZONE_KEY_CACHE = "ozone.key.cache.size";
   public static final int OZONE_KEY_CACHE_DEFAULT = 1024;
 
-  public static final int OZONE_SCM_CLIENT_PORT_DEFAULT = 9860;
-  public static final int OZONE_SCM_DATANODE_PORT_DEFAULT = 9861;
-
-  public static final String OZONE_SCM_CLIENT_ADDRESS_KEY =
-      "ozone.scm.client.address";
-  public static final String OZONE_SCM_CLIENT_BIND_HOST_KEY =
-      "ozone.scm.client.bind.host";
-  public static final String OZONE_SCM_CLIENT_BIND_HOST_DEFAULT =
-      "0.0.0.0";
-
-  public static final String OZONE_SCM_DATANODE_ADDRESS_KEY =
-      "ozone.scm.datanode.address";
-  public static final String OZONE_SCM_DATANODE_BIND_HOST_KEY =
-      "ozone.scm.datanode.bind.host";
-  public static final String OZONE_SCM_DATANODE_BIND_HOST_DEFAULT =
-      "0.0.0.0";
-
-  public static final String OZONE_SCM_HANDLER_COUNT_KEY =
-      "ozone.scm.handler.count.key";
-  public static final int OZONE_SCM_HANDLER_COUNT_DEFAULT = 10;
-
-  public static final String OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS =
-      "ozone.scm.heartbeat.interval.seconds";
-  public static final int OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT =
-      30;
-
-  public static final String OZONE_SCM_DEADNODE_INTERVAL_MS =
-      "ozone.scm.dead.node.interval.ms";
-  public static final long OZONE_SCM_DEADNODE_INTERVAL_DEFAULT =
-      OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT * 1000L * 20L;
-
-  public static final String OZONE_SCM_MAX_HB_COUNT_TO_PROCESS =
-      "ozone.scm.max.hb.count.to.process";
-  public static final int OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT = 5000;
-
-  public static final String OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS =
-      "ozone.scm.heartbeat.thread.interval.ms";
-  public static final long OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS_DEFAULT =
-      3000;
-
-  public static final String OZONE_SCM_STALENODE_INTERVAL_MS =
-      "ozone.scm.stale.node.interval.ms";
-  public static final long OZONE_SCM_STALENODE_INTERVAL_DEFAULT =
-      OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT * 1000L * 3L;
-
-  public static final String OZONE_SCM_HEARTBEAT_RPC_TIMEOUT =
-      "ozone.scm.heartbeat.rpc-timeout";
-  public static final long OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT =
-      100;
-
-  /**
-   * Defines how frequently we will log the missing of heartbeat to a specific
-   * SCM. In the default case we will write a warning message for each 10
-   * sequential heart beats that we miss to a specific SCM. This is to avoid
-   * overrunning the log with lots of HB missed Log statements.
-   */
-  public static final String OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT =
-      "ozone.scm.heartbeat.log.warn.interval.count";
-  public static final int OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT =
-      10;
-
   public static final String OZONE_CONTAINER_TASK_WAIT =
       "ozone.container.task.wait.seconds";
   public static final long OZONE_CONTAINER_TASK_WAIT_DEFAULT = 5;
 
-
-  // ozone.scm.names key is a set of DNS | DNS:PORT | IP Address | IP:PORT.
-  // Written as a comma separated string. e.g. scm1, scm2:8020, 7.7.7.7:7777
-  //
-  // If this key is not specified datanodes will not be able to find
-  // SCM. The SCM membership can be dynamic, so this key should contain
-  // all possible SCM names. Once the SCM leader is discovered datanodes will
-  // get the right list of SCMs to heartbeat to from the leader.
-  // While it is good for the datanodes to know the names of all SCM nodes,
-  // it is sufficient to actually know the name of on working SCM. That SCM
-  // will be able to return the information about other SCMs that are part of
-  // the SCM replicated Log.
-  //
-  //In case of a membership change, any one of the SCM machines will be
-  // able to send back a new list to the datanodes.
-  public static final String OZONE_SCM_NAMES = "ozone.scm.names";
-
-  public static final int OZONE_SCM_DEFAULT_PORT = 9862;
-  // File Name and path where datanode ID is to written to.
-  // 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_DB_CACHE_SIZE_MB =
-      "ozone.scm.db.cache.size.mb";
-  public static final int OZONE_SCM_DB_CACHE_SIZE_DEFAULT = 128;
-
   /**
    * There is no need to instantiate this class.
    */

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

@@ -21,7 +21,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.ozone.OzoneClientUtils;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
@@ -31,6 +30,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.VersionEndpointTask;
 import org.apache.hadoop.ozone.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -151,10 +151,10 @@ public class RunningDatanodeState implements DatanodeState {
    */
   private StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto
       getContainerNodeID() {
-    String dataNodeIDPath = conf.get(OzoneConfigKeys.OZONE_SCM_DATANODE_ID);
+    String dataNodeIDPath = conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID);
     if (dataNodeIDPath == null || dataNodeIDPath.isEmpty()) {
       LOG.error("A valid file path is needed for config setting {}",
-          OzoneConfigKeys.OZONE_SCM_DATANODE_ID);
+          ScmConfigKeys.OZONE_SCM_DATANODE_ID);
 
       // This is an unrecoverable error.
       this.context.setState(DatanodeStateMachine.DatanodeStates.SHUTDOWN);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java

@@ -19,12 +19,12 @@ package org.apache.hadoop.ozone.container.common.states.endpoint;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.statemachine
     .EndpointStateMachine;
 
 import org.apache.hadoop.ozone.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto;
+import org.apache.hadoop.scm.ScmConfigKeys;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -99,7 +99,7 @@ public final class RegisterEndpointTask implements
 
       // TODO : Add responses to the command Queue.
       rpcEndPoint.getEndPoint().register(dnNodeID,
-          conf.getStrings(OzoneConfigKeys.OZONE_SCM_NAMES));
+          conf.getStrings(ScmConfigKeys.OZONE_SCM_NAMES));
       EndpointStateMachine.EndPointStates nextState =
           rpcEndPoint.getState().getNextState();
       rpcEndPoint.setState(nextState);

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java

@@ -84,17 +84,17 @@ import java.util.UUID;
 import java.util.Map;
 import java.util.HashMap;
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_CLIENT_ADDRESS_KEY;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_DATANODE_ADDRESS_KEY;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_DB_CACHE_SIZE_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_DB_CACHE_SIZE_MB;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_HANDLER_COUNT_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_HANDLER_COUNT_KEY;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 

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

@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolPB;
 import org.apache.hadoop.ozone.scm.StorageContainerManager;
@@ -298,15 +299,15 @@ public class MiniOzoneCluster extends MiniDFSCluster implements Closeable {
       configureSCMheartbeat();
       configScmMetadata();
 
-      conf.set(OzoneConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
-      conf.set(OzoneConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
+      conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
+      conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
 
 
       StorageContainerManager scm = new StorageContainerManager(conf);
       scm.start();
       String addressString =  scm.getDatanodeRpcAddress().getHostString() +
           ":" + scm.getDatanodeRpcAddress().getPort();
-      conf.setStrings(OzoneConfigKeys.OZONE_SCM_NAMES, addressString);
+      conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, addressString);
 
       MiniOzoneCluster cluster = new MiniOzoneCluster(this, scm);
       try {
@@ -344,7 +345,7 @@ public class MiniOzoneCluster extends MiniDFSCluster implements Closeable {
       // TODO : Fix this, we need a more generic mechanism to map
       // different datanode ID for different datanodes when we have lots of
       // datanodes in the cluster.
-      conf.setStrings(OzoneConfigKeys.OZONE_SCM_DATANODE_ID,
+      conf.setStrings(ScmConfigKeys.OZONE_SCM_DATANODE_ID,
           scmPath.toString() + "/datanode.id");
 
     }
@@ -368,19 +369,19 @@ public class MiniOzoneCluster extends MiniDFSCluster implements Closeable {
 
     private void configureSCMheartbeat() {
       if (hbSeconds.isPresent()) {
-        conf.setInt(OzoneConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS,
+        conf.setInt(ScmConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS,
             hbSeconds.get());
 
       } else {
-        conf.setInt(OzoneConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS,
+        conf.setInt(ScmConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS,
             defaultHBSeconds);
       }
 
       if (hbProcessorInterval.isPresent()) {
-        conf.setInt(OzoneConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS,
+        conf.setInt(ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS,
             hbProcessorInterval.get());
       } else {
-        conf.setInt(OzoneConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS,
+        conf.setInt(ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS,
             defaultProcessorMs);
       }
 

+ 46 - 46
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestOzoneClientUtils.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.scm.ScmConfigKeys;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
@@ -30,7 +31,6 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys.*;
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
@@ -67,14 +67,14 @@ public class TestOzoneClientUtils {
 
     // First try a client address with just a host name. Verify it falls
     // back to the default port.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
     InetSocketAddress addr = OzoneClientUtils.getScmAddressForClients(conf);
     assertThat(addr.getHostString(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(OZONE_SCM_CLIENT_PORT_DEFAULT));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
 
     // Next try a client address with a host name and port. Verify both
     // are used correctly.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
     addr = OzoneClientUtils.getScmAddressForClients(conf);
     assertThat(addr.getHostString(), is("1.2.3.4"));
     assertThat(addr.getPort(), is(100));
@@ -102,31 +102,31 @@ public class TestOzoneClientUtils {
 
     // First try a client address with just a host name. Verify it falls
     // back to the default port.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
     InetSocketAddress addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
     assertThat(addr.getHostString(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(OZONE_SCM_DATANODE_PORT_DEFAULT));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
 
     // Next try a client address with just a host name and port. Verify the port
     // is ignored and the default DataNode port is used.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
     addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
     assertThat(addr.getHostString(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(OZONE_SCM_DATANODE_PORT_DEFAULT));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
 
     // Set both OZONE_SCM_CLIENT_ADDRESS_KEY and OZONE_SCM_DATANODE_ADDRESS_KEY.
     // Verify that the latter overrides and the port number is still the default.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(OZONE_SCM_DATANODE_ADDRESS_KEY, "5.6.7.8");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "5.6.7.8");
     addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
     assertThat(addr.getHostString(), is("5.6.7.8"));
-    assertThat(addr.getPort(), is(OZONE_SCM_DATANODE_PORT_DEFAULT));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
 
     // Set both OZONE_SCM_CLIENT_ADDRESS_KEY and OZONE_SCM_DATANODE_ADDRESS_KEY.
     // Verify that the latter overrides and the port number from the latter is
     // used.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(OZONE_SCM_DATANODE_ADDRESS_KEY, "5.6.7.8:200");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "5.6.7.8:200");
     addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
     assertThat(addr.getHostString(), is("5.6.7.8"));
     assertThat(addr.getPort(), is(200));
@@ -142,16 +142,16 @@ public class TestOzoneClientUtils {
 
     // The bind host should be 0.0.0.0 unless OZONE_SCM_CLIENT_BIND_HOST_KEY
     // is set differently.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
     InetSocketAddress addr = OzoneClientUtils.getScmClientBindAddress(conf);
     assertThat(addr.getHostString(), is("0.0.0.0"));
-    assertThat(addr.getPort(), is(OZONE_SCM_CLIENT_PORT_DEFAULT));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
 
     // The bind host should be 0.0.0.0 unless OZONE_SCM_CLIENT_BIND_HOST_KEY
     // is set differently. The port number from OZONE_SCM_CLIENT_ADDRESS_KEY
     // should be respected.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
     addr = OzoneClientUtils.getScmClientBindAddress(conf);
     assertThat(addr.getHostString(), is("0.0.0.0"));
     assertThat(addr.getPort(), is(100));
@@ -159,19 +159,19 @@ public class TestOzoneClientUtils {
     // OZONE_SCM_CLIENT_BIND_HOST_KEY should be respected.
     // Port number should be default if none is specified via
     // OZONE_SCM_DATANODE_ADDRESS_KEY.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
-    conf.set(OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4");
-    conf.set(OZONE_SCM_CLIENT_BIND_HOST_KEY, "5.6.7.8");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_KEY, "5.6.7.8");
     addr = OzoneClientUtils.getScmClientBindAddress(conf);
     assertThat(addr.getHostString(), is("5.6.7.8"));
-    assertThat(addr.getPort(), is(OZONE_SCM_CLIENT_PORT_DEFAULT));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
 
     // OZONE_SCM_CLIENT_BIND_HOST_KEY should be respected.
     // Port number from OZONE_SCM_CLIENT_ADDRESS_KEY should be
     // respected.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
-    conf.set(OZONE_SCM_CLIENT_BIND_HOST_KEY, "5.6.7.8");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_KEY, "5.6.7.8");
     addr = OzoneClientUtils.getScmClientBindAddress(conf);
     assertThat(addr.getHostString(), is("5.6.7.8"));
     assertThat(addr.getPort(), is(100));
@@ -187,16 +187,16 @@ public class TestOzoneClientUtils {
 
     // The bind host should be 0.0.0.0 unless OZONE_SCM_DATANODE_BIND_HOST_KEY
     // is set differently.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
     InetSocketAddress addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
     assertThat(addr.getHostString(), is("0.0.0.0"));
-    assertThat(addr.getPort(), is(OZONE_SCM_DATANODE_PORT_DEFAULT));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
 
     // The bind host should be 0.0.0.0 unless OZONE_SCM_DATANODE_BIND_HOST_KEY
     // is set differently. The port number from OZONE_SCM_DATANODE_ADDRESS_KEY
     // should be respected.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
     addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
     assertThat(addr.getHostString(), is("0.0.0.0"));
     assertThat(addr.getPort(), is(200));
@@ -204,19 +204,19 @@ public class TestOzoneClientUtils {
     // OZONE_SCM_DATANODE_BIND_HOST_KEY should be respected.
     // Port number should be default if none is specified via
     // OZONE_SCM_DATANODE_ADDRESS_KEY.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4");
-    conf.set(OZONE_SCM_DATANODE_BIND_HOST_KEY, "5.6.7.8");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_KEY, "5.6.7.8");
     addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
     assertThat(addr.getHostString(), is("5.6.7.8"));
-    assertThat(addr.getPort(), is(OZONE_SCM_DATANODE_PORT_DEFAULT));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
 
     // OZONE_SCM_DATANODE_BIND_HOST_KEY should be respected.
     // Port number from OZONE_SCM_DATANODE_ADDRESS_KEY should be
     // respected.
-    conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
-    conf.set(OZONE_SCM_DATANODE_BIND_HOST_KEY, "5.6.7.8");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_KEY, "5.6.7.8");
     addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
     assertThat(addr.getHostString(), is("5.6.7.8"));
     assertThat(addr.getPort(), is(200));
@@ -230,23 +230,23 @@ public class TestOzoneClientUtils {
     Iterator<InetSocketAddress> it = null;
 
     // Verify valid IP address setup
-    conf.setStrings(OZONE_SCM_NAMES, "1.2.3.4");
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "1.2.3.4");
     addresses = OzoneClientUtils.getSCMAddresses(conf);
     assertThat(addresses.size(), is(1));
     addr = addresses.iterator().next();
     assertThat(addr.getHostName(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(OZONE_SCM_DEFAULT_PORT));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DEFAULT_PORT));
 
     // Verify valid hostname setup
-    conf.setStrings(OZONE_SCM_NAMES, "scm1");
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1");
     addresses = OzoneClientUtils.getSCMAddresses(conf);
     assertThat(addresses.size(), is(1));
     addr = addresses.iterator().next();
     assertThat(addr.getHostName(), is("scm1"));
-    assertThat(addr.getPort(), is(OZONE_SCM_DEFAULT_PORT));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DEFAULT_PORT));
 
     // Verify valid hostname and port
-    conf.setStrings(OZONE_SCM_NAMES, "scm1:1234");
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1:1234");
     addresses = OzoneClientUtils.getSCMAddresses(conf);
     assertThat(addresses.size(), is(1));
     addr = addresses.iterator().next();
@@ -260,7 +260,7 @@ public class TestOzoneClientUtils {
     hostsAndPorts.put("scm3", 3456);
 
     // Verify multiple hosts and port
-    conf.setStrings(OZONE_SCM_NAMES, "scm1:1234,scm2:2345,scm3:3456");
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1:1234,scm2:2345,scm3:3456");
     addresses = OzoneClientUtils.getSCMAddresses(conf);
     assertThat(addresses.size(), is(3));
     it = addresses.iterator();
@@ -273,7 +273,7 @@ public class TestOzoneClientUtils {
     assertTrue(expected1.isEmpty());
 
     // Verify names with spaces
-    conf.setStrings(OZONE_SCM_NAMES, " scm1:1234, scm2:2345 , scm3:3456 ");
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, " scm1:1234, scm2:2345 , scm3:3456 ");
     addresses = OzoneClientUtils.getSCMAddresses(conf);
     assertThat(addresses.size(), is(3));
     it = addresses.iterator();
@@ -286,7 +286,7 @@ public class TestOzoneClientUtils {
     assertTrue(expected2.isEmpty());
 
     // Verify empty value
-    conf.setStrings(OZONE_SCM_NAMES, "");
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "");
     try {
       addresses = OzoneClientUtils.getSCMAddresses(conf);
       fail("Empty value should cause an IllegalArgumentException");
@@ -295,7 +295,7 @@ public class TestOzoneClientUtils {
     }
 
     // Verify invalid hostname
-    conf.setStrings(OZONE_SCM_NAMES, "s..x..:1234");
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "s..x..:1234");
     try {
       addresses = OzoneClientUtils.getSCMAddresses(conf);
       fail("An invalid hostname should cause an IllegalArgumentException");
@@ -304,7 +304,7 @@ public class TestOzoneClientUtils {
     }
 
     // Verify invalid port
-    conf.setStrings(OZONE_SCM_NAMES, "scm:xyz");
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm:xyz");
     try {
       addresses = OzoneClientUtils.getSCMAddresses(conf);
       fail("An invalid port should cause an IllegalArgumentException");
@@ -313,7 +313,7 @@ public class TestOzoneClientUtils {
     }
 
     // Verify a mixed case (valid and invalid value both appears)
-    conf.setStrings(OZONE_SCM_NAMES, "scm1:1234, scm:xyz");
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1:1234, scm:xyz");
     try {
       addresses = OzoneClientUtils.getSCMAddresses(conf);
       fail("An invalid value should cause an IllegalArgumentException");

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

@@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.container.common;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
@@ -49,7 +49,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
+import static org.apache.hadoop.scm.ScmConfigKeys
     .OZONE_SCM_HEARTBEAT_RPC_TIMEOUT;
 
 /**
@@ -83,7 +83,7 @@ public class TestDatanodeStateMachine {
       mockServers.add(mock);
     }
 
-    conf.setStrings(OzoneConfigKeys.OZONE_SCM_NAMES,
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES,
         serverAddresses.toArray(new String[0]));
 
     URL p = this.getClass().getResource("");
@@ -96,7 +96,7 @@ public class TestDatanodeStateMachine {
     conf.set(DFS_DATANODE_DATA_DIR_KEY, path);
     path = Paths.get(path.toString(),
         TestDatanodeStateMachine.class.getSimpleName() + ".id").toString();
-    conf.set(OzoneConfigKeys.OZONE_SCM_DATANODE_ID, path);
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ID, path);
     executorService = HadoopExecutors.newCachedThreadPool(
         new ThreadFactoryBuilder().setDaemon(true)
             .setNameFormat("Test Data Node State Machine Thread - %d").build());
@@ -275,7 +275,7 @@ public class TestDatanodeStateMachine {
         "scm:xyz",   // Invalid port
         "scm:123456" // Port out of range
     }) {
-      conf.setStrings(OzoneConfigKeys.OZONE_SCM_NAMES, name);
+      conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, name);
       final DatanodeStateMachine stateMachine =
           new DatanodeStateMachine(conf);
       DatanodeStateMachine.DatanodeStates currentState =

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

@@ -19,13 +19,13 @@ package org.apache.hadoop.ozone.scm.node;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
 import org.apache.hadoop.ozone.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
 import org.apache.hadoop.ozone.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;
@@ -40,11 +40,11 @@ import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.TimeoutException;
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_DEADNODE_INTERVAL_MS;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_STALENODE_INTERVAL_MS;
+import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL_MS;
+import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS;
+import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS;
+import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS;
+import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL_MS;
 import static org.apache.hadoop.ozone.scm.node.NodeManager.NODESTATE.HEALTHY;
 import static org.apache.hadoop.ozone.scm.node.NodeManager.NODESTATE.STALE;
 import static org.apache.hadoop.ozone.scm.node.NodeManager.NODESTATE.DEAD;
@@ -202,7 +202,7 @@ public class TestNodeManager {
   public void testScmShutdown() throws IOException, InterruptedException,
       TimeoutException {
     Configuration conf = getConf();
-    conf.setInt(OzoneConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS, 100);
+    conf.setInt(ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS, 100);
     SCMNodeManager nodeManager = createNodeManager(conf);
     DatanodeID datanodeID = SCMTestUtils.getDatanodeID(nodeManager);
     nodeManager.close();