瀏覽代碼

Fix merge conflicts

Xiaoyu Yao 6 年之前
父節點
當前提交
570b503e3e

+ 0 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java

@@ -436,8 +436,6 @@ public abstract class TestConfigurationFieldsBase {
     // Create XML key/value map
     // Create XML key/value map
     LOG_XML.debug("Reading XML property files\n");
     LOG_XML.debug("Reading XML property files\n");
     xmlKeyValueMap = extractPropertiesFromXml(xmlFilename);
     xmlKeyValueMap = extractPropertiesFromXml(xmlFilename);
-    // Remove hadoop property set in ozone-default.xml
-    xmlKeyValueMap.remove("hadoop.custom.tags");
     LOG_XML.debug("\n=====\n");
     LOG_XML.debug("\n=====\n");
 
 
     // Create default configuration variable key/value map
     // Create default configuration variable key/value map

+ 6 - 4
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java

@@ -326,10 +326,12 @@ public final class ScmConfigKeys {
   public static final String HDDS_SCM_WATCHER_TIMEOUT_DEFAULT =
   public static final String HDDS_SCM_WATCHER_TIMEOUT_DEFAULT =
       "10m";
       "10m";
 
 
-  public static final String SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY =
-      "ozone.scm.web.authentication.kerberos.principal";
-  public static final String SCM_WEB_AUTHENTICATION_KERBEROS_KEYTAB_FILE_KEY =
-      "ozone.scm.web.authentication.kerberos.keytab";
+  public static final String
+      HDDS_SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY =
+      "hdds.scm.web.authentication.kerberos.principal";
+  public static final String
+      HDDS_SCM_WEB_AUTHENTICATION_KERBEROS_KEYTAB_FILE_KEY =
+      "hdds.scm.web.authentication.kerberos.keytab";
   /**
   /**
    * Never constructed.
    * Never constructed.
    */
    */

+ 1 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java

@@ -352,6 +352,7 @@ public final class OzoneConfigKeys {
 
 
   public static final String OZONE_CONTAINER_COPY_WORKDIR =
   public static final String OZONE_CONTAINER_COPY_WORKDIR =
       "hdds.datanode.replication.work.dir";
       "hdds.datanode.replication.work.dir";
+
   /**
   /**
    * Config properties to set client side checksum properties.
    * Config properties to set client side checksum properties.
    */
    */

+ 19 - 1
hadoop-hdds/common/src/main/resources/ozone-default.xml

@@ -529,6 +529,14 @@
       ozone.om.http-address.
       ozone.om.http-address.
     </description>
     </description>
   </property>
   </property>
+  <property>
+    <name>ozone.om.keytab.file</name>
+    <value/>
+    <tag>OM, SECURITY</tag>
+    <description>
+      The keytab file for Kerberos authentication in OM.
+    </description>
+  </property>
   <property>
   <property>
     <name>ozone.om.db.cache.size.mb</name>
     <name>ozone.om.db.cache.size.mb</name>
     <value>128</value>
     <value>128</value>
@@ -1041,7 +1049,7 @@
   </property>
   </property>
 
 
   <property>
   <property>
-    <name>hdds.scm.container.creation.lease.timeout</name>
+    <name>ozone.scm.container.creation.lease.timeout</name>
     <value>60s</value>
     <value>60s</value>
     <tag>OZONE, SCM</tag>
     <tag>OZONE, SCM</tag>
     <description>
     <description>
@@ -1300,6 +1308,7 @@
       datanode unless the datanode confirms the completion.
       datanode unless the datanode confirms the completion.
     </description>
     </description>
   </property>
   </property>
+
   <property>
   <property>
     <name>hdds.db.profile</name>
     <name>hdds.db.profile</name>
     <value>DISK</value>
     <value>DISK</value>
@@ -1308,6 +1317,7 @@
     that tunes the RocksDB settings for the hardware it is running
     that tunes the RocksDB settings for the hardware it is running
     on. Right now, we have SSD and DISK as profile options.</description>
     on. Right now, we have SSD and DISK as profile options.</description>
   </property>
   </property>
+
   <property>
   <property>
     <name>hdds.datanode.replication.work.dir</name>
     <name>hdds.datanode.replication.work.dir</name>
     <tag>DATANODE</tag>
     <tag>DATANODE</tag>
@@ -1409,6 +1419,14 @@
       file. Unit could be defined with postfix (ns,ms,s,m,h,d)
       file. Unit could be defined with postfix (ns,ms,s,m,h,d)
     </description>
     </description>
   </property>
   </property>
+  <property>
+    <name>ozone.security.enabled</name>
+    <value>false</value>
+    <tag> OZONE, SECURITY, FLAG</tag>
+    <description>True if security is enabled for ozone. When this property is
+     true, hadoop.security.authentication should be Kerberos.
+    </description>
+  </property>
 
 
   <property>
   <property>
     <name>ozone.client.checksum.type</name>
     <name>ozone.client.checksum.type</name>

+ 5 - 4
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java

@@ -449,11 +449,12 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
    * @param args command line arguments.
    * @param args command line arguments.
    * @param conf HDDS configuration
    * @param conf HDDS configuration
    * @return SCM instance
    * @return SCM instance
-   * @throws IOException
+   * @throws IOException, AuthenticationException
    */
    */
   @VisibleForTesting
   @VisibleForTesting
-  public static StorageContainerManager createSCM(String[] args,
-      OzoneConfiguration conf) throws IOException, AuthenticationException {
+  public static StorageContainerManager createSCM(
+      String[] args, OzoneConfiguration conf)
+      throws IOException, AuthenticationException {
     return createSCM(args, conf, false);
     return createSCM(args, conf, false);
   }
   }
 
 
@@ -464,7 +465,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
    * @param conf HDDS configuration
    * @param conf HDDS configuration
    * @param printBanner if true, then log a verbose startup message.
    * @param printBanner if true, then log a verbose startup message.
    * @return SCM instance
    * @return SCM instance
-   * @throws IOException
+   * @throws IOException, AuthenticationException
    */
    */
   private static StorageContainerManager createSCM(
   private static StorageContainerManager createSCM(
       String[] args,
       String[] args,

+ 2 - 2
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerHttpServer.java

@@ -62,11 +62,11 @@ public class StorageContainerManagerHttpServer extends BaseHttpServer {
   }
   }
 
 
   @Override protected String getKeytabFile() {
   @Override protected String getKeytabFile() {
-    return ScmConfigKeys.SCM_WEB_AUTHENTICATION_KERBEROS_KEYTAB_FILE_KEY;
+    return ScmConfigKeys.HDDS_SCM_WEB_AUTHENTICATION_KERBEROS_KEYTAB_FILE_KEY;
   }
   }
 
 
   @Override protected String getSpnegoPrincipal() {
   @Override protected String getSpnegoPrincipal() {
-    return ScmConfigKeys.SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
+    return ScmConfigKeys.HDDS_SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
   }
   }
 
 
   @Override protected String getEnabledKey() {
   @Override protected String getEnabledKey() {

+ 1 - 1
hadoop-ozone/acceptance-test/src/test/compose/compose-secure/.env

@@ -14,4 +14,4 @@
 # See the License for the specific language governing permissions and
 # See the License for the specific language governing permissions and
 # limitations under the License.
 # limitations under the License.
 
 
-OZONEDIR=../../../hadoop-dist/target/ozone
+OZONEDIR=../../../../../../hadoop-dist/target/ozone-0.3.0-SNAPSHOT

+ 44 - 11
hadoop-ozone/acceptance-test/src/test/compose/compose-secure/docker-config

@@ -22,7 +22,7 @@ OZONE-SITE.XML_hdds.scm.block.client.address=scm
 OZONE-SITE.XML_ozone.metadata.dirs=/data/metadata
 OZONE-SITE.XML_ozone.metadata.dirs=/data/metadata
 OZONE-SITE.XML_ozone.handler.type=distributed
 OZONE-SITE.XML_ozone.handler.type=distributed
 OZONE-SITE.XML_hdds.scm.client.address=scm
 OZONE-SITE.XML_hdds.scm.client.address=scm
-OZONE-SITE.XML_hdds.datanode.plugins=org.apache.hadoop.ozone.web.OzoneHddsDatanodeService
+OZONE-SITE.XML_ozone.replication=1
 OZONE-SITE.XML_hdds.scm.kerberos.principal=scm/scm@EXAMPLE.COM
 OZONE-SITE.XML_hdds.scm.kerberos.principal=scm/scm@EXAMPLE.COM
 OZONE-SITE.XML_hdds.scm.kerberos.keytab.file=/etc/security/keytabs/scm.keytab
 OZONE-SITE.XML_hdds.scm.kerberos.keytab.file=/etc/security/keytabs/scm.keytab
 OZONE-SITE.XML_ozone.om.kerberos.principal=om/om@EXAMPLE.COM
 OZONE-SITE.XML_ozone.om.kerberos.principal=om/om@EXAMPLE.COM
@@ -32,21 +32,12 @@ OZONE-SITE.XML_hdds.scm.web.authentication.kerberos.principal=HTTP/scm@EXAMPLE.C
 OZONE-SITE.XML_hdds.scm.web.authentication.kerberos.keytab=/etc/security/keytabs/HTTP.keytab
 OZONE-SITE.XML_hdds.scm.web.authentication.kerberos.keytab=/etc/security/keytabs/HTTP.keytab
 OZONE-SITE.XML_ozone.om.web.authentication.kerberos.principal=HTTP/om@EXAMPLE.COM
 OZONE-SITE.XML_ozone.om.web.authentication.kerberos.principal=HTTP/om@EXAMPLE.COM
 OZONE-SITE.XML_ozone.om.web.authentication.kerberos.keytab=/etc/security/keytabs/HTTP.keytab
 OZONE-SITE.XML_ozone.om.web.authentication.kerberos.keytab=/etc/security/keytabs/HTTP.keytab
-OZONE-SITE.XML_ozone.scm.block.client.address=scm
-OZONE-SITE.XML_ozone.scm.client.address=scm
-HDFS-SITE.XML_dfs.namenode.name.dir=/data/namenode
-HDFS-SITE.XML_dfs.datanode.plugins=org.apache.hadoop.ozone.HddsDatanodeService
-HDFS-SITE.XML_dfs.block.access.token.enable=true
-HDFS-SITE.XML_dfs.namenode.kerberos.principal=nn/namenode@EXAMPLE.COM
-HDFS-SITE.XML_dfs.namenode.keytab.file=/etc/security/keytabs/nn.keytab
 HDFS-SITE.XML_dfs.datanode.kerberos.principal=dn/datanode@EXAMPLE.COM
 HDFS-SITE.XML_dfs.datanode.kerberos.principal=dn/datanode@EXAMPLE.COM
 HDFS-SITE.XML_dfs.datanode.keytab.file=/etc/security/keytabs/dn.keytab
 HDFS-SITE.XML_dfs.datanode.keytab.file=/etc/security/keytabs/dn.keytab
-HDFS-SITE.XML_dfs.namenode.kerberos.internal.spnego.principal=HTTP/namenode@EXAMPLE.COM
 HDFS-SITE.XML_dfs.web.authentication.kerberos.principal=HTTP/_HOST@EXAMPLE.COM
 HDFS-SITE.XML_dfs.web.authentication.kerberos.principal=HTTP/_HOST@EXAMPLE.COM
 HDFS-SITE.XML_dfs.web.authentication.kerberos.keytab=/etc/security/keytabs/HTTP.keytab
 HDFS-SITE.XML_dfs.web.authentication.kerberos.keytab=/etc/security/keytabs/HTTP.keytab
 HDFS-SITE.XML_dfs.datanode.address=0.0.0.0:1019
 HDFS-SITE.XML_dfs.datanode.address=0.0.0.0:1019
 HDFS-SITE.XML_dfs.datanode.http.address=0.0.0.0:1012
 HDFS-SITE.XML_dfs.datanode.http.address=0.0.0.0:1012
-HDFS-SITE.XML_dfs.namenode.rpc-address=namenode:9000
 CORE-SITE.XML_dfs.data.transfer.protection=authentication
 CORE-SITE.XML_dfs.data.transfer.protection=authentication
 CORE-SITE.XML_hadoop.security.authentication=kerberos
 CORE-SITE.XML_hadoop.security.authentication=kerberos
 CORE-SITE.XML_hadoop.security.auth_to_local=RULE:[2:$1@$0](.*)s/.*/root/
 CORE-SITE.XML_hadoop.security.auth_to_local=RULE:[2:$1@$0](.*)s/.*/root/
@@ -55,9 +46,51 @@ LOG4J.PROPERTIES_log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 LOG4J.PROPERTIES_log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 LOG4J.PROPERTIES_log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 LOG4J.PROPERTIES_log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
 LOG4J.PROPERTIES_log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
 
 
+
+#Enable this variable to print out all hadoop rpc traffic to the stdout. See http://byteman.jboss.org/ to define your own instrumentation.
+#BYTEMAN_SCRIPT_URL=https://raw.githubusercontent.com/apache/hadoop/trunk/dev-support/byteman/hadooprpc.btm
+
+#LOG4J2.PROPERTIES_* are for Ozone Audit Logging
+LOG4J2.PROPERTIES_monitorInterval=30
+LOG4J2.PROPERTIES_filter=read,write
+LOG4J2.PROPERTIES_filter.read.type=MarkerFilter
+LOG4J2.PROPERTIES_filter.read.marker=READ
+LOG4J2.PROPERTIES_filter.read.onMatch=DENY
+LOG4J2.PROPERTIES_filter.read.onMismatch=NEUTRAL
+LOG4J2.PROPERTIES_filter.write.type=MarkerFilter
+LOG4J2.PROPERTIES_filter.write.marker=WRITE
+LOG4J2.PROPERTIES_filter.write.onMatch=NEUTRAL
+LOG4J2.PROPERTIES_filter.write.onMismatch=NEUTRAL
+LOG4J2.PROPERTIES_appenders=console, rolling
+LOG4J2.PROPERTIES_appender.console.type=Console
+LOG4J2.PROPERTIES_appender.console.name=STDOUT
+LOG4J2.PROPERTIES_appender.console.layout.type=PatternLayout
+LOG4J2.PROPERTIES_appender.console.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
+LOG4J2.PROPERTIES_appender.rolling.type=RollingFile
+LOG4J2.PROPERTIES_appender.rolling.name=RollingFile
+LOG4J2.PROPERTIES_appender.rolling.fileName =${sys:hadoop.log.dir}/om-audit-${hostName}.log
+LOG4J2.PROPERTIES_appender.rolling.filePattern=${sys:hadoop.log.dir}/om-audit-${hostName}-%d{yyyy-MM-dd-HH-mm-ss}-%i.log.gz
+LOG4J2.PROPERTIES_appender.rolling.layout.type=PatternLayout
+LOG4J2.PROPERTIES_appender.rolling.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
+LOG4J2.PROPERTIES_appender.rolling.policies.type=Policies
+LOG4J2.PROPERTIES_appender.rolling.policies.time.type=TimeBasedTriggeringPolicy
+LOG4J2.PROPERTIES_appender.rolling.policies.time.interval=86400
+LOG4J2.PROPERTIES_appender.rolling.policies.size.type=SizeBasedTriggeringPolicy
+LOG4J2.PROPERTIES_appender.rolling.policies.size.size=64MB
+LOG4J2.PROPERTIES_loggers=audit
+LOG4J2.PROPERTIES_logger.audit.type=AsyncLogger
+LOG4J2.PROPERTIES_logger.audit.name=OMAudit
+LOG4J2.PROPERTIES_logger.audit.level=INFO
+LOG4J2.PROPERTIES_logger.audit.appenderRefs=rolling
+LOG4J2.PROPERTIES_logger.audit.appenderRef.file.ref=RollingFile
+LOG4J2.PROPERTIES_rootLogger.level=INFO
+LOG4J2.PROPERTIES_rootLogger.appenderRefs=stdout
+LOG4J2.PROPERTIES_rootLogger.appenderRef.stdout.ref=STDOUT
+
+
 OZONE_DATANODE_SECURE_USER=root
 OZONE_DATANODE_SECURE_USER=root
 CONF_DIR=/etc/security/keytabs
 CONF_DIR=/etc/security/keytabs
-KERBEROS_KEYTABS=dn nn om scm HTTP testuser
+KERBEROS_KEYTABS=dn om scm HTTP testuser
 KERBEROS_KEYSTORES=hadoop
 KERBEROS_KEYSTORES=hadoop
 KERBEROS_SERVER=ozone.kdc
 KERBEROS_SERVER=ozone.kdc
 JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64/
 JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64/

+ 0 - 84
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java

@@ -1,84 +0,0 @@
-/**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.ksm;
-
-import org.apache.hadoop.ozone.OzoneAcl;
-/**
- * KSM Constants.
- */
-public final class KSMConfigKeys {
-  /**
-   * Never constructed.
-   */
-  private KSMConfigKeys() {
-  }
-
-
-  public static final String OZONE_KSM_HANDLER_COUNT_KEY =
-      "ozone.ksm.handler.count.key";
-  public static final int OZONE_KSM_HANDLER_COUNT_DEFAULT = 20;
-
-  public static final String OZONE_KSM_ADDRESS_KEY =
-      "ozone.ksm.address";
-  public static final String OZONE_KSM_BIND_HOST_DEFAULT =
-      "0.0.0.0";
-  public static final int OZONE_KSM_PORT_DEFAULT = 9862;
-
-  public static final String OZONE_KSM_HTTP_ENABLED_KEY =
-      "ozone.ksm.http.enabled";
-  public static final String OZONE_KSM_HTTP_BIND_HOST_KEY =
-      "ozone.ksm.http-bind-host";
-  public static final String OZONE_KSM_HTTPS_BIND_HOST_KEY =
-      "ozone.ksm.https-bind-host";
-  public static final String OZONE_KSM_HTTP_ADDRESS_KEY =
-      "ozone.ksm.http-address";
-  public static final String OZONE_KSM_HTTPS_ADDRESS_KEY =
-      "ozone.ksm.https-address";
-  public static final String OZONE_KSM_HTTP_BIND_HOST_DEFAULT = "0.0.0.0";
-  public static final int OZONE_KSM_HTTP_BIND_PORT_DEFAULT = 9874;
-  public static final int OZONE_KSM_HTTPS_BIND_PORT_DEFAULT = 9875;
-
-  // LevelDB cache file uses an off-heap cache in LevelDB of 128 MB.
-  public static final String OZONE_KSM_DB_CACHE_SIZE_MB =
-      "ozone.ksm.db.cache.size.mb";
-  public static final int OZONE_KSM_DB_CACHE_SIZE_DEFAULT = 128;
-
-  public static final String OZONE_KSM_USER_MAX_VOLUME =
-      "ozone.ksm.user.max.volume";
-  public static final int OZONE_KSM_USER_MAX_VOLUME_DEFAULT = 1024;
-
-  // KSM Default user/group permissions
-  public static final String OZONE_KSM_USER_RIGHTS =
-      "ozone.ksm.user.rights";
-  public static final OzoneAcl.OzoneACLRights OZONE_KSM_USER_RIGHTS_DEFAULT =
-      OzoneAcl.OzoneACLRights.READ_WRITE;
-
-  public static final String OZONE_KSM_GROUP_RIGHTS =
-      "ozone.ksm.group.rights";
-  public static final OzoneAcl.OzoneACLRights OZONE_KSM_GROUP_RIGHTS_DEFAULT =
-      OzoneAcl.OzoneACLRights.READ_WRITE;
-
-  public static final String OZONE_KEY_DELETING_LIMIT_PER_TASK =
-      "ozone.key.deleting.limit.per.task";
-  public static final int OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT = 1000;
-
-  public static final String OZONE_OM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL =
-      "ozone.om.web.authentication.kerberos.principal";
-  public static final String OZONE_OM_WEB_AUTHENTICATION_KERBEROS_KEYTAB_FILE =
-      "ozone.om.web.authentication.kerberos.keytab";
-}

+ 2 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java

@@ -150,7 +150,8 @@ public interface MiniOzoneCluster {
    * @throws TimeoutException
    * @throws TimeoutException
    * @throws InterruptedException
    * @throws InterruptedException
    */
    */
-  void restartStorageContainerManager() throws InterruptedException, TimeoutException, IOException, AuthenticationException;
+  void restartStorageContainerManager() throws InterruptedException,
+      TimeoutException, IOException, AuthenticationException;
 
 
   /**
   /**
    * Restarts OzoneManager instance.
    * Restarts OzoneManager instance.

+ 3 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java

@@ -233,8 +233,9 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
   }
   }
 
 
   @Override
   @Override
-  public void restartStorageContainerManager() throws TimeoutException,
-      InterruptedException, IOException, AuthenticationException {
+  public void restartStorageContainerManager()
+      throws TimeoutException, InterruptedException, IOException,
+      AuthenticationException {
     scm.stop();
     scm.stop();
     scm.join();
     scm.join();
     scm = StorageContainerManager.createSCM(null, conf);
     scm = StorageContainerManager.createSCM(null, conf);

+ 12 - 10
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java

@@ -91,10 +91,6 @@ public final class TestSecureOzoneCluster {
       startMiniKdc();
       startMiniKdc();
       setSecureConfig(conf);
       setSecureConfig(conf);
       createCredentialsInKDC(conf, miniKdc);
       createCredentialsInKDC(conf, miniKdc);
-
-      clusterId = UUID.randomUUID().toString();
-      scmId = UUID.randomUUID().toString();
-      omId = UUID.randomUUID().toString();
     } catch (IOException e) {
     } catch (IOException e) {
       LOGGER.error("Failed to initialize TestSecureOzoneCluster", e);
       LOGGER.error("Failed to initialize TestSecureOzoneCluster", e);
     } catch (Exception e) {
     } catch (Exception e) {
@@ -122,11 +118,12 @@ public final class TestSecureOzoneCluster {
     createPrincipal(scmKeytab,
     createPrincipal(scmKeytab,
         conf.get(ScmConfigKeys.HDDS_SCM_KERBEROS_PRINCIPAL_KEY));
         conf.get(ScmConfigKeys.HDDS_SCM_KERBEROS_PRINCIPAL_KEY));
      createPrincipal(spnegoKeytab,
      createPrincipal(spnegoKeytab,
-         conf.get(ScmConfigKeys.SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY),
-         conf.get(OMConfigKeys.OZONE_OM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY));
-    createPrincipal(omKeyTab,
+         conf.get(ScmConfigKeys
+             .HDDS_SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY));
         conf.get(OMConfigKeys
         conf.get(OMConfigKeys
-            .OZONE_OM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY));
+            .OZONE_OM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY);
+    createPrincipal(omKeyTab,
+        conf.get(OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY));
   }
   }
 
 
   private void createPrincipal(File keytab, String... principal)
   private void createPrincipal(File keytab, String... principal)
@@ -158,7 +155,7 @@ public final class TestSecureOzoneCluster {
 
 
     conf.set(ScmConfigKeys.HDDS_SCM_KERBEROS_PRINCIPAL_KEY,
     conf.set(ScmConfigKeys.HDDS_SCM_KERBEROS_PRINCIPAL_KEY,
         "scm/" + host + "@" + realm);
         "scm/" + host + "@" + realm);
-    conf.set(ScmConfigKeys.SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY,
+    conf.set(ScmConfigKeys.HDDS_SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY,
         "HTTP_SCM/" + host + "@" + realm);
         "HTTP_SCM/" + host + "@" + realm);
 
 
     conf.set(OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY,
     conf.set(OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY,
@@ -172,7 +169,7 @@ public final class TestSecureOzoneCluster {
 
 
     conf.set(ScmConfigKeys.HDDS_SCM_KERBEROS_KEYTAB_FILE_KEY,
     conf.set(ScmConfigKeys.HDDS_SCM_KERBEROS_KEYTAB_FILE_KEY,
         scmKeytab.getAbsolutePath());
         scmKeytab.getAbsolutePath());
-    conf.set(ScmConfigKeys.SCM_WEB_AUTHENTICATION_KERBEROS_KEYTAB_FILE_KEY,
+    conf.set(ScmConfigKeys.HDDS_SCM_WEB_AUTHENTICATION_KERBEROS_KEYTAB_FILE_KEY,
         spnegoKeytab.getAbsolutePath());
         spnegoKeytab.getAbsolutePath());
     conf.set(OMConfigKeys.OZONE_OM_KERBEROS_KEYTAB_FILE_KEY,
     conf.set(OMConfigKeys.OZONE_OM_KERBEROS_KEYTAB_FILE_KEY,
         omKeyTab.getAbsolutePath());
         omKeyTab.getAbsolutePath());
@@ -191,6 +188,11 @@ public final class TestSecureOzoneCluster {
 
 
   private void initSCM()
   private void initSCM()
       throws IOException, AuthenticationException {
       throws IOException, AuthenticationException {
+
+    clusterId = UUID.randomUUID().toString();
+    scmId = UUID.randomUUID().toString();
+    omId = UUID.randomUUID().toString();
+
     final String path = GenericTestUtils
     final String path = GenericTestUtils
         .getTempPath(UUID.randomUUID().toString());
         .getTempPath(UUID.randomUUID().toString());
     Path scmPath = Paths.get(path, "scm-meta");
     Path scmPath = Paths.get(path, "scm-meta");