You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by rl...@apache.org on 2016/10/12 09:01:23 UTC

ambari git commit: AMBARI-18564. Ambari should be able to create arbitrary Kerberos identities for itself as declared in the Kerberos Descriptor (rlevas)

Repository: ambari
Updated Branches:
  refs/heads/trunk 1d54fefbb -> d2a926ee7


AMBARI-18564. Ambari should be able to create arbitrary Kerberos identities for itself as declared in the Kerberos Descriptor (rlevas)


Project: http://git-wip-us.apache.org/repos/asf/ambari/repo
Commit: http://git-wip-us.apache.org/repos/asf/ambari/commit/d2a926ee
Tree: http://git-wip-us.apache.org/repos/asf/ambari/tree/d2a926ee
Diff: http://git-wip-us.apache.org/repos/asf/ambari/diff/d2a926ee

Branch: refs/heads/trunk
Commit: d2a926ee71f8a24a8d568fc977dc8ac06dc18493
Parents: 1d54fef
Author: Robert Levas <rl...@hortonworks.com>
Authored: Wed Oct 12 05:01:14 2016 -0400
Committer: Robert Levas <rl...@hortonworks.com>
Committed: Wed Oct 12 05:01:14 2016 -0400

----------------------------------------------------------------------
 .../server/controller/KerberosHelper.java       |  40 +++++-
 .../server/controller/KerberosHelperImpl.java   | 131 +++++++++----------
 .../AbstractPrepareKerberosServerAction.java    |  38 +++---
 .../ConfigureAmbariIdentitiesServerAction.java  |   2 +-
 .../resources/stacks/HDP/2.0.6/kerberos.json    |  40 ++++--
 .../server/controller/KerberosHelperTest.java   |  84 ++++++++----
 6 files changed, 196 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/d2a926ee/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
index 1153d01..35235db 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
@@ -65,15 +65,24 @@ public interface KerberosHelper {
    * The alias to assign to the KDC administrator credential Keystore item
    */
   String KDC_ADMINISTRATOR_CREDENTIAL_ALIAS = "kdc.admin.credential";
-
+  /**
+   * The hostname used to hold the place of the actual hostname of the host that the Ambari server
+   * is on.
+   */
   String AMBARI_SERVER_HOST_NAME = "ambari_server";
-
-  String AMBARI_IDENTITY_NAME = "ambari-server";
-
-  String SPNEGO_IDENTITY_NAME = "spnego";
-
+  /**
+   * The name of the Ambari server's Kerberos identities as defined in the Kerberos descriptor
+   */
+  String AMBARI_SERVER_KERBEROS_IDENTITY_NAME = "ambari-server";
+  /**
+   * The kerberos-env property name declaring whether Ambari should manage its own required
+   * identities or not
+   */
   String CREATE_AMBARI_PRINCIPAL = "create_ambari_principal";
-
+  /**
+   * The kerberos-env property name declaring whether Ambari should manage the cluster's required
+   * identities or not
+   */
   String MANAGE_IDENTITIES = "manage_identities";
 
   /**
@@ -607,6 +616,23 @@ public interface KerberosHelper {
       throws AmbariException;
 
   /**
+   * Gets the Ambari server Kerberos identities found in the Kerberos descriptor.
+   *
+   * @param kerberosDescriptor the kerberos descriptor
+   */
+  List<KerberosIdentityDescriptor> getAmbariServerIdentities(KerberosDescriptor kerberosDescriptor) throws AmbariException;
+
+  /**
+   * Determines if the Ambari identities should be created when enabling Kerberos.
+   * <p>
+   * If kerberos-env/create_ambari_principal is not set to false the identity should be calculated.
+   *
+   * @param kerberosEnvProperties the kerberos-env configuration properties
+   * @return true if the Ambari identities should be created; otherwise false
+   */
+  boolean createAmbariIdentities(Map<String, String> kerberosEnvProperties);
+
+  /**
    * Gets the previously stored KDC administrator credentials.
    *
    * @param clusterName the name of the relevant cluster

http://git-wip-us.apache.org/repos/asf/ambari/blob/d2a926ee/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelperImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelperImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelperImpl.java
index e15965b..fb87495 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelperImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelperImpl.java
@@ -359,6 +359,13 @@ public class KerberosHelperImpl implements KerberosHelper {
 
     Map<String, Set<String>> propertiesToIgnore = new HashMap<String, Set<String>>();
 
+    // If Ambari is managing it own identities then add AMBARI to the set of installed servcie so
+    // that its Kerberos descriptor entries will be included.
+    if(createAmbariIdentities(existingConfigurations.get("kerberos-env"))) {
+      installedServices = new HashMap<String, Set<String>>(installedServices);
+      installedServices.put("AMBARI", Collections.singleton("AMBARI_SERVER"));
+    }
+
     // Create the context to use for filtering Kerberos Identities based on the state of the cluster
     Map<String, Object> filterContext = new HashMap<String, Object>();
     filterContext.put("configurations", configurations);
@@ -401,28 +408,6 @@ public class KerberosHelperImpl implements KerberosHelper {
       }
     }
 
-    if (kerberosDetails.createAmbariPrincipal()) {
-      KerberosIdentityDescriptor ambariServerIdentityDescriptor = kerberosDescriptor.getIdentity(KerberosHelper.AMBARI_IDENTITY_NAME);
-      Map<String, Map<String, String>> map = new HashMap<String, Map<String, String>>();
-      if (ambariServerIdentityDescriptor != null) {
-
-        KerberosPrincipalDescriptor principalDescriptor = ambariServerIdentityDescriptor.getPrincipalDescriptor();
-        if (principalDescriptor != null) {
-          putConfiguration(map, principalDescriptor.getConfiguration(), principalDescriptor.getValue());
-        }
-
-        KerberosKeytabDescriptor keytabDescriptor = ambariServerIdentityDescriptor.getKeytabDescriptor();
-        if (keytabDescriptor != null) {
-          putConfiguration(map, keytabDescriptor.getConfiguration(), keytabDescriptor.getFile());
-        }
-
-        for (Map.Entry<String, Map<String, String>> entry : map.entrySet()) {
-          String configType = entry.getKey();
-          mergeConfigurations(kerberosConfigurations, configType, entry.getValue(), configurations);
-        }
-      }
-    }
-
     setAuthToLocalRules(kerberosDescriptor, kerberosDetails.getDefaultRealm(), installedServices, configurations, kerberosConfigurations);
 
     return (applyStackAdvisorUpdates)
@@ -764,41 +749,37 @@ public class KerberosHelperImpl implements KerberosHelper {
    *                                 configuration
    * @throws AmbariException
    */
-  private void installAmbariIdentities(AbstractKerberosDescriptorContainer kerberosDescriptor,
+  private void installAmbariIdentities(KerberosDescriptor kerberosDescriptor,
                                        KerberosOperationHandler kerberosOperationHandler,
                                        Map<String, String> kerberosEnvProperties,
                                        Map<String, Map<String, String>> configurations,
                                        KerberosDetails kerberosDetails) throws AmbariException {
 
-    // Install Ambari's user/service principal...
-    String ambariServerHostname = StageUtils.getHostName();
-    KerberosIdentityDescriptor identity;
+    // Install Ambari's identities.....
+    List<KerberosIdentityDescriptor> ambariIdentities = getAmbariServerIdentities(kerberosDescriptor);
 
-    identity = kerberosDescriptor.getIdentity(KerberosHelper.AMBARI_IDENTITY_NAME);
-    if (identity != null) {
-      KerberosPrincipalDescriptor principal = identity.getPrincipalDescriptor();
-      if (principal != null) {
-        Keytab keytab = createIdentity(identity, principal.getType(), kerberosEnvProperties, kerberosOperationHandler, configurations, ambariServerHostname);
-        installAmbariIdentity(identity, keytab, configurations, ambariServerHostname, kerberosDetails, true);
+    if (!ambariIdentities.isEmpty()) {
+      String ambariServerHostname = StageUtils.getHostName();
 
-        try {
-          KerberosChecker.checkJaasConfiguration();
-        } catch (AmbariException e) {
-          LOG.error("Error in Ambari JAAS configuration: " + e.getLocalizedMessage(), e);
+      for (KerberosIdentityDescriptor identity : ambariIdentities) {
+        if (identity != null) {
+          KerberosPrincipalDescriptor principal = identity.getPrincipalDescriptor();
+          if (principal != null) {
+            boolean updateJAASFile = AMBARI_SERVER_KERBEROS_IDENTITY_NAME.equals(identity.getName());
+            Keytab keytab = createIdentity(identity, principal.getType(), kerberosEnvProperties, kerberosOperationHandler, configurations, ambariServerHostname);
+            installAmbariIdentity(identity, keytab, configurations, ambariServerHostname, kerberosDetails, updateJAASFile);
+
+            if (updateJAASFile) {
+              try {
+                KerberosChecker.checkJaasConfiguration();
+              } catch (AmbariException e) {
+                LOG.error("Error in Ambari JAAS configuration: " + e.getLocalizedMessage(), e);
+              }
+            }
+          }
         }
       }
     }
-
-    // Install Ambari's SPNGEO principal...
-    identity = kerberosDescriptor.getIdentity(KerberosHelper.SPNEGO_IDENTITY_NAME);
-    if (identity != null) {
-      KerberosPrincipalDescriptor principal = identity.getPrincipalDescriptor();
-
-      if (principal != null) {
-        Keytab keytab = createIdentity(identity, principal.getType(), kerberosEnvProperties, kerberosOperationHandler, configurations, ambariServerHostname);
-        installAmbariIdentity(identity, keytab, configurations, ambariServerHostname, kerberosDetails, false);
-      }
-    }
   }
 
   /**
@@ -1351,7 +1332,14 @@ public class KerberosHelperImpl implements KerberosHelper {
                 serviceName, componentName, kerberosDescriptor, filterContext);
 
             if (hostname.equals(ambariServerHostname)) {
-              addAmbariServerIdentities(kerberosEnvConfig.getProperties(), kerberosDescriptor, identities);
+              // Determine if we should _calculate_ the Ambari service identities.
+              // If kerberos-env/create_ambari_principal is not set to false the identity should be calculated.
+              if(createAmbariIdentities(kerberosEnvConfig.getProperties())) {
+                List<KerberosIdentityDescriptor> ambariIdentities = getAmbariServerIdentities(kerberosDescriptor);
+                if (ambariIdentities != null) {
+                  identities.addAll(ambariIdentities);
+                }
+              }
             }
 
             if (!identities.isEmpty()) {
@@ -1427,35 +1415,34 @@ public class KerberosHelperImpl implements KerberosHelper {
     return activeIdentities;
   }
 
-  /**
-   * Conditionally add the Ambari server Kerberos identity to the set of Kerberos Identities expected
-   * to be available when Kerberos is enabled.
-   * <p>
-   * The Ambari server Kerberos identity should only be added if the <code>kerberos-env/create_ambari_principal</code>
-   * property is not explicitly set to <code>false</code>.
-   *
-   * @param kerberosEnvProperties the kerberos-env properties
-   * @param kerberosDescriptor    the kerberos descriptor
-   * @param identities            the collection of identities to add to
-   */
-  void addAmbariServerIdentities(Map<String, String> kerberosEnvProperties, KerberosDescriptor kerberosDescriptor, List<KerberosIdentityDescriptor> identities) {
-    // Determine if we should _calculate_ the Ambari service identity.
-    // If kerberos-env/create_ambari_principal is not set to false the identity should be calculated.
-    boolean createAmbariPrincipal = (kerberosEnvProperties == null) || !"false".equalsIgnoreCase(kerberosEnvProperties.get(CREATE_AMBARI_PRINCIPAL));
+  @Override
+  public List<KerberosIdentityDescriptor> getAmbariServerIdentities(KerberosDescriptor kerberosDescriptor) throws AmbariException {
+    List<KerberosIdentityDescriptor> ambariIdentities = new ArrayList<KerberosIdentityDescriptor>();
 
-    // append Ambari server principal
-    if (createAmbariPrincipal) {
-      KerberosIdentityDescriptor ambariServerIdentity = kerberosDescriptor.getIdentity(KerberosHelper.AMBARI_IDENTITY_NAME);
-      if (ambariServerIdentity != null) {
-        identities.add(ambariServerIdentity);
+    KerberosServiceDescriptor ambariKerberosDescriptor = kerberosDescriptor.getService("AMBARI");
+    if (ambariKerberosDescriptor != null) {
+      List<KerberosIdentityDescriptor> serviceIdentities = ambariKerberosDescriptor.getIdentities(true, null);
+      KerberosComponentDescriptor ambariServerKerberosComponentDescriptor = ambariKerberosDescriptor.getComponent("AMBARI_SERVER");
+
+      if (serviceIdentities != null) {
+        ambariIdentities.addAll(serviceIdentities);
       }
 
-      // Add the spnego principal for the Ambari server host....
-      KerberosIdentityDescriptor spnegoIdentity = kerberosDescriptor.getIdentity(KerberosHelper.SPNEGO_IDENTITY_NAME);
-      if (spnegoIdentity != null) {
-        identities.add(spnegoIdentity);
+      if (ambariServerKerberosComponentDescriptor != null) {
+        List<KerberosIdentityDescriptor> componentIdentities = ambariServerKerberosComponentDescriptor.getIdentities(true, null);
+
+        if (componentIdentities != null) {
+          ambariIdentities.addAll(componentIdentities);
+        }
       }
     }
+
+    return ambariIdentities;
+  }
+
+  @Override
+  public boolean createAmbariIdentities(Map<String, String> kerberosEnvProperties) {
+    return (kerberosEnvProperties == null) || !"false".equalsIgnoreCase(kerberosEnvProperties.get(CREATE_AMBARI_PRINCIPAL));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/ambari/blob/d2a926ee/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java
index c283a65..7a2fa40 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java
@@ -147,28 +147,26 @@ public abstract class AbstractPrepareKerberosServerAction extends KerberosServer
             }
           }
 
-          // Add ambari-server principal (and keytab) only if 'kerberos-env.create_ambari_principal = true'
-          Map<String, String> kerberosEnvProperties = configurations.get("kerberos-env");
-          if (kerberosEnvProperties != null && kerberosEnvProperties.get(KerberosHelper.CREATE_AMBARI_PRINCIPAL) != null
-            && "true".equalsIgnoreCase(kerberosEnvProperties.get(KerberosHelper.CREATE_AMBARI_PRINCIPAL))
-              && includeAmbariIdentity) {
-            KerberosIdentityDescriptor ambariServerIdentity = kerberosDescriptor.getIdentity(KerberosHelper.AMBARI_IDENTITY_NAME);
-            if (ambariServerIdentity != null) {
-              List<KerberosIdentityDescriptor> componentIdentities = Collections.singletonList(ambariServerIdentity);
-              kerberosHelper.addIdentities(kerberosIdentityDataFileWriter, componentIdentities,
-                identityFilter, KerberosHelper.AMBARI_SERVER_HOST_NAME, "AMBARI_SERVER", "AMBARI_SERVER", kerberosConfigurations, configurations);
-              propertiesToIgnore = gatherPropertiesToIgnore(componentIdentities, propertiesToIgnore);
-            }
-
-            KerberosIdentityDescriptor spnegoIdentity = kerberosDescriptor.getIdentity(KerberosHelper.SPNEGO_IDENTITY_NAME);
-            if (spnegoIdentity != null) {
-              List<KerberosIdentityDescriptor> componentIdentities = Collections.singletonList(spnegoIdentity);
-              kerberosHelper.addIdentities(kerberosIdentityDataFileWriter, componentIdentities,
-                identityFilter, KerberosHelper.AMBARI_SERVER_HOST_NAME, "AMBARI_SERVER", "SPNEGO", kerberosConfigurations, configurations);
-              propertiesToIgnore = gatherPropertiesToIgnore(componentIdentities, propertiesToIgnore);
+          // Add ambari-server identities only if 'kerberos-env.create_ambari_principal = true'
+          if (includeAmbariIdentity && kerberosHelper.createAmbariIdentities(configurations.get("kerberos-env"))) {
+            List<KerberosIdentityDescriptor> ambariIdentities = kerberosHelper.getAmbariServerIdentities(kerberosDescriptor);
+
+            if (!ambariIdentities.isEmpty()) {
+              for (KerberosIdentityDescriptor identity : ambariIdentities) {
+                // If the identity represents the ambari-server user, use the component name "AMBARI_SERVER_SELF"
+                // so it can be distinguished between other identities related to the AMBARI-SERVER
+                // component.
+                String componentName = KerberosHelper.AMBARI_SERVER_KERBEROS_IDENTITY_NAME.equals(identity.getName())
+                    ? "AMBARI_SERVER_SELF"
+                    : "AMBARI_SERVER";
+
+                List<KerberosIdentityDescriptor> componentIdentities = Collections.singletonList(identity);
+                kerberosHelper.addIdentities(kerberosIdentityDataFileWriter, componentIdentities,
+                    identityFilter, KerberosHelper.AMBARI_SERVER_HOST_NAME, "AMBARI", componentName, kerberosConfigurations, configurations);
+                propertiesToIgnore = gatherPropertiesToIgnore(componentIdentities, propertiesToIgnore);
+              }
             }
           }
-
         } catch (IOException e) {
           String message = String.format("Failed to write index file - %s", identityDataFile.getAbsolutePath());
           LOG.error(message, e);

http://git-wip-us.apache.org/repos/asf/ambari/blob/d2a926ee/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/ConfigureAmbariIdentitiesServerAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/ConfigureAmbariIdentitiesServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/ConfigureAmbariIdentitiesServerAction.java
index 80acd0d..aa846ea 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/ConfigureAmbariIdentitiesServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/ConfigureAmbariIdentitiesServerAction.java
@@ -137,7 +137,7 @@ public class ConfigureAmbariIdentitiesServerAction extends KerberosServerAction
                 identityRecord.get(KerberosIdentityDataFileReader.KEYTAB_FILE_OWNER_NAME), ownerReadable, ownerWritable,
                 identityRecord.get(KerberosIdentityDataFileReader.KEYTAB_FILE_GROUP_NAME), groupReadable, groupWritable, actionLog);
 
-            if ("AMBARI_SERVER".equals(identityRecord.get(KerberosIdentityDataFileReader.COMPONENT))) {
+            if ("AMBARI_SERVER_SELF".equals(identityRecord.get(KerberosIdentityDataFileReader.COMPONENT))) {
               // Create/update the JAASFile...
               configureJAAS(evaluatedPrincipal, destKeytabFilePath, actionLog);
             }

http://git-wip-us.apache.org/repos/asf/ambari/blob/d2a926ee/ambari-server/src/main/resources/stacks/HDP/2.0.6/kerberos.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/kerberos.json b/ambari-server/src/main/resources/stacks/HDP/2.0.6/kerberos.json
index 0644f57..9579d0f 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/kerberos.json
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/kerberos.json
@@ -43,20 +43,36 @@
         },
         "configuration": "cluster-env/smokeuser_keytab"
       }
-    },
+    }
+  ],
+  "services": [
     {
-      "name": "ambari-server",
-      "principal": {
-        "value": "ambari-server-${cluster_name|toLower()}@${realm}",
-        "type": "user",
-        "configuration": "cluster-env/ambari_principal_name"
-      },
-      "keytab": {
-        "file": "${keytab_dir}/ambari.server.keytab",
-        "owner": {
-          "access": "r"
+      "name": "AMBARI",
+      "components": [
+        {
+          "name": "AMBARI_SERVER",
+          "identities": [
+            {
+              "name": "ambari-server",
+              "principal": {
+                "value": "ambari-server-${cluster_name|toLower()}@${realm}",
+                "type": "user",
+                "configuration": "cluster-env/ambari_principal_name"
+              },
+              "keytab": {
+                "file": "${keytab_dir}/ambari.server.keytab",
+                "owner": {
+                  "access": "r"
+                }
+              }
+            },
+            {
+              "name" : "ambari-server_spnego",
+              "reference" : "/spnego"
+            }
+          ]
         }
-      }
+      ]
     }
   ]
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/d2a926ee/ambari-server/src/test/java/org/apache/ambari/server/controller/KerberosHelperTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/KerberosHelperTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/KerberosHelperTest.java
index b2eb738..bd23e00 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/KerberosHelperTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/KerberosHelperTest.java
@@ -2455,7 +2455,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     testEnsureHeadlessIdentities(true, true);
   }
 
-  private void testEnsureHeadlessIdentities(boolean createAmbariPrincipal, boolean ambariServerPrincipalAsService) throws Exception {
+  private void testEnsureHeadlessIdentities(boolean createAmbariIdentities, boolean ambariServerPrincipalAsService) throws Exception {
     String clusterName = "c1";
     String realm = "EXAMPLE.COM";
     String ambariServerHostname = StageUtils.getHostName();
@@ -2486,7 +2486,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     propertiesKerberosEnv.put("password_min_digits", "1");
     propertiesKerberosEnv.put("password_min_punctuation", "0");
     propertiesKerberosEnv.put("password_min_whitespace", "0");
-    propertiesKerberosEnv.put("create_ambari_principal", (createAmbariPrincipal) ? "true" : "false");
+    propertiesKerberosEnv.put("create_ambari_principal", (createAmbariIdentities) ? "true" : "false");
 
     Config configKrb5Conf = createMock(Config.class);
     expect(configKrb5Conf.getProperties()).andReturn(propertiesKrb5Conf).times(1);
@@ -2576,21 +2576,29 @@ public class KerberosHelperTest extends EasyMockSupport {
     expect(kerberosDescriptor.getService("SERVICE1")).andReturn(service1KerberosDescriptor).times(1);
     expect(kerberosDescriptor.getService("SERVICE2")).andReturn(service2KerberosDescriptor).times(1);
 
-    if (createAmbariPrincipal) {
+    if (createAmbariIdentities) {
       String spnegoPrincipalNameExpected = String.format("HTTP/%s@%s", ambariServerHostname, realm);
 
-      KerberosIdentityDescriptor ambariIdentity = createMockIdentityDescriptor(
-          KerberosHelper.AMBARI_IDENTITY_NAME,
+      ArrayList<KerberosIdentityDescriptor> ambarServerComponent1Identities = new ArrayList<KerberosIdentityDescriptor>();
+      ambarServerComponent1Identities.add(createMockIdentityDescriptor(
+          KerberosHelper.AMBARI_SERVER_KERBEROS_IDENTITY_NAME,
           createMockPrincipalDescriptor(ambariServerPrincipalName, ambariServerPrincipalType, "ambari", null),
-          createMockKeytabDescriptor(ambariServerKeytabFilePath, null));
+          createMockKeytabDescriptor(ambariServerKeytabFilePath, null)));
 
-      KerberosIdentityDescriptor spnegoIdentity = createMockIdentityDescriptor(
-          KerberosHelper.SPNEGO_IDENTITY_NAME,
+      ambarServerComponent1Identities.add(createMockIdentityDescriptor(
+          "spnego",
           createMockPrincipalDescriptor("HTTP/_HOST@${realm}", KerberosPrincipalType.SERVICE, null, null),
-          createMockKeytabDescriptor("spnego.service.keytab", null));
+          createMockKeytabDescriptor("spnego.service.keytab", null)));
+
+      KerberosComponentDescriptor ambariServerComponentKerberosDescriptor = createMockComponentDescriptor("AMBARI_SERVER", ambarServerComponent1Identities, null);
+
+      HashMap<String, KerberosComponentDescriptor> ambariServerComponentDescriptorMap = new HashMap<String, KerberosComponentDescriptor>();
+      ambariServerComponentDescriptorMap.put("AMBARI_SERVER", ambariServerComponentKerberosDescriptor);
+
+      KerberosServiceDescriptor ambariServiceKerberosDescriptor = createMockServiceDescriptor("AMBARI", ambariServerComponentDescriptorMap, null);
+      expect(ambariServiceKerberosDescriptor.getComponent("AMBARI_SERVER")).andReturn(ambariServerComponentKerberosDescriptor).once();
 
-      expect(kerberosDescriptor.getIdentity(KerberosHelper.AMBARI_IDENTITY_NAME)).andReturn(ambariIdentity).once();
-      expect(kerberosDescriptor.getIdentity(KerberosHelper.SPNEGO_IDENTITY_NAME)).andReturn(spnegoIdentity).once();
+      expect(kerberosDescriptor.getService("AMBARI")).andReturn(ambariServiceKerberosDescriptor).once();
 
       ConfigureAmbariIdentitiesServerAction configureAmbariIdentitiesServerAction = injector.getInstance(ConfigureAmbariIdentitiesServerAction.class);
       expect(configureAmbariIdentitiesServerAction.installAmbariServerIdentity(eq(ambariServerPrincipalNameExpected), anyString(), eq(ambariServerKeytabFilePath),
@@ -2626,7 +2634,7 @@ public class KerberosHelperTest extends EasyMockSupport {
         .andReturn(new CreatePrincipalsServerAction.CreatePrincipalResult("anything", "password", 1))
         .times(3);
 
-    if(createAmbariPrincipal) {
+    if(createAmbariIdentities) {
       if (ambariServerPrincipalAsService) {
         expect(createPrincipalsServerAction.createPrincipal(capture(capturePrincipal), eq(true), anyObject(Map.class), anyObject(KerberosOperationHandler.class), eq(false), isNull(ActionLog.class)))
             .andReturn(new CreatePrincipalsServerAction.CreatePrincipalResult("anything", "password", 1))
@@ -2644,7 +2652,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     CreateKeytabFilesServerAction createKeytabFilesServerAction = injector.getInstance(CreateKeytabFilesServerAction.class);
     expect(createKeytabFilesServerAction.createKeytab(capture(capturePrincipalForKeytab), eq("password"), eq(1), anyObject(KerberosOperationHandler.class), eq(true), eq(true), isNull(ActionLog.class)))
         .andReturn(new Keytab())
-        .times(createAmbariPrincipal ? 5 : 3);
+        .times(createAmbariIdentities ? 5 : 3);
 
     replayAll();
 
@@ -2661,18 +2669,18 @@ public class KerberosHelperTest extends EasyMockSupport {
     verifyAll();
 
     List<? extends String> capturedPrincipals = capturePrincipal.getValues();
-    assertEquals(createAmbariPrincipal ? 5 : 3, capturedPrincipals.size());
+    assertEquals(createAmbariIdentities ? 5 : 3, capturedPrincipals.size());
     assertTrue(capturedPrincipals.contains("s1_1@EXAMPLE.COM"));
     assertTrue(capturedPrincipals.contains("s1c1_1@EXAMPLE.COM"));
     assertTrue(capturedPrincipals.contains("s2_1@EXAMPLE.COM"));
 
     List<? extends String> capturedPrincipalsForKeytab = capturePrincipalForKeytab.getValues();
-    assertEquals(createAmbariPrincipal ? 5 : 3, capturedPrincipalsForKeytab.size());
+    assertEquals(createAmbariIdentities ? 5 : 3, capturedPrincipalsForKeytab.size());
     assertTrue(capturedPrincipalsForKeytab.contains("s1_1@EXAMPLE.COM"));
     assertTrue(capturedPrincipalsForKeytab.contains("s1c1_1@EXAMPLE.COM"));
     assertTrue(capturedPrincipalsForKeytab.contains("s2_1@EXAMPLE.COM"));
 
-    if(createAmbariPrincipal) {
+    if(createAmbariIdentities) {
       String spnegoPrincipalName = String.format("HTTP/%s@EXAMPLE.COM", ambariServerHostname);
 
       assertTrue(capturedPrincipals.contains(ambariServerPrincipalNameExpected));
@@ -3861,31 +3869,53 @@ public class KerberosHelperTest extends EasyMockSupport {
 
   private void addAmbariServerIdentity(Map<String, String> kerberosEnvProperties) throws Exception {
 
-    boolean createAmbariPrincipal = (kerberosEnvProperties == null)
-        || !"false".equalsIgnoreCase(kerberosEnvProperties.get("create_ambari_principal"));
-
     KerberosHelperImpl kerberosHelper = injector.getInstance(KerberosHelperImpl.class);
 
-    KerberosIdentityDescriptor ambariKerberosIdentity = createMock(KerberosIdentityDescriptor.class);
+    boolean createAmbariIdentities = kerberosHelper.createAmbariIdentities(kerberosEnvProperties);
+
+    KerberosIdentityDescriptor ambariKerberosIdentity = null;
 
     KerberosDescriptor kerberosDescriptor = createMock(KerberosDescriptor.class);
-    if (createAmbariPrincipal) {
-      expect(kerberosDescriptor.getIdentity(KerberosHelper.AMBARI_IDENTITY_NAME)).andReturn(ambariKerberosIdentity).once();
-      expect(kerberosDescriptor.getIdentity(KerberosHelper.SPNEGO_IDENTITY_NAME)).andReturn(ambariKerberosIdentity).once();
-    }
+    if (createAmbariIdentities) {
+      String ambariServerPrincipalName = "ambari-server-${cluster_name}@${realm}";
+      KerberosPrincipalType ambariServerPrincipalType = KerberosPrincipalType.USER;
+      String ambariServerKeytabFilePath = new File("ambari.server.keytab").getAbsolutePath();
+
+      ambariKerberosIdentity = createMockIdentityDescriptor(
+          KerberosHelper.AMBARI_SERVER_KERBEROS_IDENTITY_NAME,
+          createMockPrincipalDescriptor(ambariServerPrincipalName, ambariServerPrincipalType, "ambari", null),
+          createMockKeytabDescriptor(ambariServerKeytabFilePath, null));
+
+      ArrayList<KerberosIdentityDescriptor> ambarServerComponent1Identities = new ArrayList<KerberosIdentityDescriptor>();
+      ambarServerComponent1Identities.add(ambariKerberosIdentity);
+
+      ambarServerComponent1Identities.add(createMockIdentityDescriptor(
+          "ambari-server_spnego",
+          createMockPrincipalDescriptor("HTTP/_HOST@${realm}", KerberosPrincipalType.SERVICE, null, null),
+          createMockKeytabDescriptor("spnego.service.keytab", null)));
+
+      HashMap<String, KerberosComponentDescriptor> ambariServerComponentDescriptorMap = new HashMap<String, KerberosComponentDescriptor>();
+      KerberosComponentDescriptor componentDescrptor = createMockComponentDescriptor("AMBARI_SERVER", ambarServerComponent1Identities, null);
+      ambariServerComponentDescriptorMap.put("AMBARI_SERVER", componentDescrptor);
 
-    List<KerberosIdentityDescriptor> identities = new ArrayList<KerberosIdentityDescriptor>();
+      KerberosServiceDescriptor ambariServiceKerberosDescriptor = createMockServiceDescriptor("AMBARI", ambariServerComponentDescriptorMap, null);
+      expect(ambariServiceKerberosDescriptor.getComponent("AMBARI_SERVER")).andReturn(componentDescrptor).once();
+
+      expect(kerberosDescriptor.getService("AMBARI")).andReturn(ambariServiceKerberosDescriptor).once();
+    }
 
     replayAll();
 
     // Needed by infrastructure
     injector.getInstance(AmbariMetaInfo.class).init();
 
-    kerberosHelper.addAmbariServerIdentities(kerberosEnvProperties, kerberosDescriptor, identities);
+    List<KerberosIdentityDescriptor> identities = (createAmbariIdentities)
+        ? kerberosHelper.getAmbariServerIdentities(kerberosDescriptor)
+        : new ArrayList<KerberosIdentityDescriptor>();
 
     verifyAll();
 
-    if (createAmbariPrincipal) {
+    if (createAmbariIdentities) {
       Assert.assertEquals(2, identities.size());
       Assert.assertSame(ambariKerberosIdentity, identities.get(0));
     } else {