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/04 17:16:18 UTC

[1/2] ambari git commit: AMBARI-18463. Regression: krb5JAASLogin.conf is not updated during secure BP install (rlevas)

Repository: ambari
Updated Branches:
  refs/heads/trunk 269494849 -> 1bf206907


http://git-wip-us.apache.org/repos/asf/ambari/blob/1bf20690/ambari-server/src/test/java/org/apache/ambari/server/serveraction/kerberos/ConfigureAmbariIdentitiesServerActionTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/serveraction/kerberos/ConfigureAmbariIdentitiesServerActionTest.java b/ambari-server/src/test/java/org/apache/ambari/server/serveraction/kerberos/ConfigureAmbariIdentitiesServerActionTest.java
new file mode 100644
index 0000000..439bcae
--- /dev/null
+++ b/ambari-server/src/test/java/org/apache/ambari/server/serveraction/kerberos/ConfigureAmbariIdentitiesServerActionTest.java
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ambari.server.serveraction.kerberos;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import junit.framework.Assert;
+import org.apache.ambari.server.audit.AuditLogger;
+import org.apache.ambari.server.controller.KerberosHelper;
+import org.apache.ambari.server.orm.DBAccessor;
+import org.apache.ambari.server.orm.dao.HostDAO;
+import org.apache.ambari.server.orm.dao.KerberosPrincipalHostDAO;
+import org.apache.ambari.server.orm.entities.HostEntity;
+import org.apache.ambari.server.serveraction.ActionLog;
+import org.apache.ambari.server.state.Clusters;
+import org.apache.ambari.server.utils.StageUtils;
+import org.apache.commons.io.FileUtils;
+import org.easymock.EasyMockSupport;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import javax.persistence.EntityManager;
+import java.io.File;
+import java.lang.reflect.Method;
+
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+
+public class ConfigureAmbariIdentitiesServerActionTest extends EasyMockSupport {
+  @Rule
+  public TemporaryFolder testFolder = new TemporaryFolder();
+
+  @Test
+  public void installAmbariServerIdentity() throws Exception {
+    installAmbariServerIdentity(createNiceMock(ActionLog.class), true);
+  }
+
+  @Test
+  public void installAmbariServerIdentityWithNoAgentOnAmbariServer() throws Exception {
+    installAmbariServerIdentity(createNiceMock(ActionLog.class), false);
+  }
+
+  @Test
+  public void installAmbariServerIdentityWithNullActionLog() throws Exception {
+    installAmbariServerIdentity(null, true);
+  }
+
+  private void installAmbariServerIdentity(ActionLog actionLog, boolean ambariServerHasAgent) throws Exception {
+
+    String principal = "ambari-server@EXAMPLE.COM";
+    File srcKeytabFile = testFolder.newFile();
+    File destKeytabFile = new File(testFolder.getRoot().getAbsolutePath(), "ambari-server.keytab");
+
+    Injector injector = createInjector();
+
+    HostEntity hostEntity;
+
+    if (ambariServerHasAgent) {
+      KerberosPrincipalHostDAO kerberosPrincipalHostDAO = injector.getInstance(KerberosPrincipalHostDAO.class);
+      expect(kerberosPrincipalHostDAO.exists(principal, 1L)).andReturn(false).once();
+      kerberosPrincipalHostDAO.create(principal, 1L);
+      expectLastCall().once();
+
+      hostEntity = createMock(HostEntity.class);
+      expect(hostEntity.getHostId()).andReturn(1L).once();
+    } else {
+      hostEntity = null;
+    }
+
+    HostDAO hostDAO = injector.getInstance(HostDAO.class);
+    expect(hostDAO.findByName(StageUtils.getHostName())).andReturn(hostEntity).once();
+
+    // Mock the methods that do the actual file manipulation to avoid having to deal with ambari-sudo.sh used in
+    // ShellCommandUtil#mkdir, ShellCommandUtil#copyFile, etc..
+    Method methodCopyFile = ConfigureAmbariIdentitiesServerAction.class.getDeclaredMethod("copyFile",
+        String.class, String.class);
+    Method methodSetFileACL = ConfigureAmbariIdentitiesServerAction.class.getDeclaredMethod("setFileACL",
+        String.class, String.class, boolean.class, boolean.class, String.class, boolean.class, boolean.class);
+
+    ConfigureAmbariIdentitiesServerAction action = createMockBuilder(ConfigureAmbariIdentitiesServerAction.class)
+        .addMockedMethod(methodCopyFile)
+        .addMockedMethod(methodSetFileACL)
+        .createMock();
+
+    action.copyFile(srcKeytabFile.getAbsolutePath(), destKeytabFile.getAbsolutePath());
+    expectLastCall().once();
+
+    action.setFileACL(destKeytabFile.getAbsolutePath(), "user1", true, true, "groupA", true, false);
+    expectLastCall().once();
+
+    replayAll();
+
+    injector.injectMembers(action);
+    action.installAmbariServerIdentity(principal, srcKeytabFile.getAbsolutePath(), destKeytabFile.getAbsolutePath(),
+        "user1", true, true, "groupA", true, false, actionLog);
+
+    verifyAll();
+
+    // There is no need to verify that the file was copied. We are not testing the ability to copy
+    // and we have mocked the method that does the actual copying to avoid having to deal with
+    // ambari-sudo.sh via the ShellCommandUtil class.
+  }
+
+  @Test
+  public void configureJAAS() throws Exception {
+    configureJAAS(createNiceMock(ActionLog.class));
+  }
+
+  @Test
+  public void configureJAASWithNullActionLog() throws Exception {
+    configureJAAS(null);
+  }
+
+  private void configureJAAS(ActionLog actionLog) throws Exception {
+    String principal = "ambari-server@EXAMPLE.COM";
+    String keytabFilePath = "/etc/security/keytabs/ambari.server.keytab";
+
+    File jaasConfFile = testFolder.newFile();
+    File jaasConfFileBak = new File(jaasConfFile.getAbsolutePath() + ".bak");
+    String originalJAASFileContent =
+        "com.sun.security.jgss.krb5.initiate {\n" +
+            "    com.sun.security.auth.module.Krb5LoginModule required\n" +
+            "    renewTGT=false\n" +
+            "    doNotPrompt=true\n" +
+            "    useKeyTab=true\n" +
+            "    keyTab=\"/etc/security/keytabs/ambari.keytab\"\n" +
+            "    principal=\"ambari@EXAMPLE.COM\"\n" +
+            "    storeKey=true\n" +
+            "    useTicketCache=false;\n" +
+            "};\n";
+
+    FileUtils.writeStringToFile(jaasConfFile, originalJAASFileContent);
+
+    Injector injector = createInjector();
+
+    Method methodGetJAASConfFilePath = ConfigureAmbariIdentitiesServerAction.class.getDeclaredMethod("getJAASConfFilePath");
+
+    ConfigureAmbariIdentitiesServerAction action = createMockBuilder(ConfigureAmbariIdentitiesServerAction.class)
+        .addMockedMethod(methodGetJAASConfFilePath)
+        .createMock();
+
+    expect(action.getJAASConfFilePath()).andReturn(jaasConfFile.getAbsolutePath());
+
+    replayAll();
+
+    injector.injectMembers(action);
+    action.configureJAAS(principal, keytabFilePath, actionLog);
+
+    verifyAll();
+
+    Assert.assertEquals(
+        "com.sun.security.jgss.krb5.initiate {\n" +
+            "    com.sun.security.auth.module.Krb5LoginModule required\n" +
+            "    renewTGT=false\n" +
+            "    doNotPrompt=true\n" +
+            "    useKeyTab=true\n" +
+            "    keyTab=\"/etc/security/keytabs/ambari.server.keytab\"\n" +
+            "    principal=\"ambari-server@EXAMPLE.COM\"\n" +
+            "    storeKey=true\n" +
+            "    useTicketCache=false;\n" +
+            "};\n",
+        FileUtils.readFileToString(jaasConfFile)
+    );
+
+    // Ensure the backup file matches the original content
+    Assert.assertEquals(originalJAASFileContent, FileUtils.readFileToString(jaasConfFileBak));
+  }
+
+
+  private Injector createInjector() {
+    return Guice.createInjector(new AbstractModule() {
+      @Override
+      protected void configure() {
+        bind(EntityManager.class).toInstance(createNiceMock(EntityManager.class));
+        bind(DBAccessor.class).toInstance(createNiceMock(DBAccessor.class));
+        bind(AuditLogger.class).toInstance(createNiceMock(AuditLogger.class));
+        bind(Clusters.class).toInstance(createNiceMock(Clusters.class));
+        bind(KerberosHelper.class).toInstance(createNiceMock(KerberosHelper.class));
+
+        bind(HostDAO.class).toInstance(createMock(HostDAO.class));
+        bind(KerberosPrincipalHostDAO.class).toInstance(createMock(KerberosPrincipalHostDAO.class));
+      }
+    });
+  }
+
+}
\ No newline at end of file


[2/2] ambari git commit: AMBARI-18463. Regression: krb5JAASLogin.conf is not updated during secure BP install (rlevas)

Posted by rl...@apache.org.
AMBARI-18463. Regression: krb5JAASLogin.conf is not updated during secure BP install (rlevas)


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

Branch: refs/heads/trunk
Commit: 1bf206907ea26eeeada640406ae2c130aa4140c9
Parents: 2694948
Author: Robert Levas <rl...@hortonworks.com>
Authored: Tue Oct 4 13:15:43 2016 -0400
Committer: Robert Levas <rl...@hortonworks.com>
Committed: Tue Oct 4 13:16:08 2016 -0400

----------------------------------------------------------------------
 .../server/controller/KerberosHelperImpl.java   | 243 ++++++++++++++-----
 .../ConfigureAmbariIdentitiesServerAction.java  | 190 +++++++++++----
 .../kerberos/FinalizeKerberosServerAction.java  | 118 ++++++++-
 .../kerberos/KerberosOperationHandler.java      |   2 +-
 .../ambari/server/utils/ShellCommandUtil.java   | 121 +++++++++
 .../resources/stacks/HDP/2.0.6/kerberos.json    |  14 +-
 .../server/controller/KerberosHelperTest.java   | 195 ++++++++++++---
 ...nfigureAmbariIdentitiesServerActionTest.java | 204 ++++++++++++++++
 8 files changed, 936 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/1bf20690/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 a3c6fd4..cd4a0b5 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
@@ -54,7 +54,6 @@ import org.apache.ambari.server.metadata.RoleCommandOrder;
 import org.apache.ambari.server.orm.dao.ArtifactDAO;
 import org.apache.ambari.server.orm.dao.KerberosPrincipalDAO;
 import org.apache.ambari.server.orm.entities.ArtifactEntity;
-import org.apache.ambari.server.orm.entities.KerberosPrincipalEntity;
 import org.apache.ambari.server.security.credential.Credential;
 import org.apache.ambari.server.security.credential.PrincipalKeyCredential;
 import org.apache.ambari.server.security.encryption.CredentialStoreService;
@@ -133,7 +132,7 @@ public class KerberosHelperImpl implements KerberosHelper {
   /**
    * The set of states a component may be in, indicating that is have been previously installed on
    * the cluster.
-   *
+   * <p>
    * These values are important when trying to determine the state of the cluster when adding new components
    */
   private static final Set<State> PREVIOUSLY_INSTALLED_STATES = EnumSet.of(State.INSTALLED, State.STARTED, State.DISABLED);
@@ -711,7 +710,7 @@ public class KerberosHelperImpl implements KerberosHelper {
                 identityDescriptors = serviceDescriptor.getIdentities(true, filterContext);
                 if (identityDescriptors != null) {
                   for (KerberosIdentityDescriptor identityDescriptor : identityDescriptors) {
-                    createUserIdentity(identityDescriptor, kerberosConfiguration, kerberosOperationHandler, configurations);
+                    createIdentity(identityDescriptor, KerberosPrincipalType.USER, kerberosConfiguration, kerberosOperationHandler, configurations, null);
                   }
                 }
 
@@ -719,7 +718,7 @@ public class KerberosHelperImpl implements KerberosHelper {
                 identityDescriptors = componentDescriptor.getIdentities(true, filterContext);
                 if (identityDescriptors != null) {
                   for (KerberosIdentityDescriptor identityDescriptor : identityDescriptors) {
-                    createUserIdentity(identityDescriptor, kerberosConfiguration, kerberosOperationHandler, configurations);
+                    createIdentity(identityDescriptor, KerberosPrincipalType.USER, kerberosConfiguration, kerberosOperationHandler, configurations, null);
                   }
                 }
               }
@@ -730,16 +729,7 @@ public class KerberosHelperImpl implements KerberosHelper {
 
       // create Ambari principal & keytab, configure JAAS only if 'kerberos-env.create_ambari_principal = true'
       if (kerberosDetails.createAmbariPrincipal()) {
-        KerberosIdentityDescriptor ambariServerIdentity = kerberosDescriptor.getIdentity(KerberosHelper.AMBARI_IDENTITY_NAME);
-        if (ambariServerIdentity != null) {
-          createUserIdentity(ambariServerIdentity, kerberosConfiguration, kerberosOperationHandler, configurations);
-          installAmbariIdentity(ambariServerIdentity, configurations);
-          try {
-            KerberosChecker.checkJaasConfiguration();
-          } catch (AmbariException e) {
-            LOG.error("Error in Ambari JAAS configuration: ", e);
-          }
-        }
+        installAmbariIdentities(kerberosDescriptor, kerberosOperationHandler, kerberosConfiguration, configurations, kerberosDetails);
       }
 
       // The KerberosOperationHandler needs to be closed, if it fails to close ignore the
@@ -756,27 +746,129 @@ public class KerberosHelperImpl implements KerberosHelper {
   }
 
   /**
+   * Install identities needed by the Ambari server, itself.
+   * <p>
+   * The Ambari server needs its own identity for authentication; and, if Kerberos authentication is
+   * enabled, it needs a SPNEGO principal for ticket validation routines.
+   * <p>
+   * Any identities needed by the Ambari server need to be installed separately since an agent may not
+   * exist on the host and therefore distributing the keytab file(s) to the Ambari server host may
+   * not be possible using the same workflow used for other hosts in the cluster.
+   *
+   * @param kerberosDescriptor       the Kerberos descriptor
+   * @param kerberosOperationHandler the relevant KerberosOperationHandler
+   * @param kerberosEnvProperties    the kerberos-env properties
+   * @param configurations           a map of config-types to property name/value pairs representing
+   *                                 the existing configurations for the cluster
+   * @param kerberosDetails          a KerberosDetails containing information about relevant Kerberos
+   *                                 configuration
+   * @throws AmbariException
+   */
+  private void installAmbariIdentities(AbstractKerberosDescriptorContainer 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;
+
+    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);
+
+        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);
+      }
+    }
+  }
+
+  /**
    * Performs tasks needed to install the Kerberos identities created for the Ambari server.
    *
    * @param ambariServerIdentity the ambari server's {@link KerberosIdentityDescriptor}
-   * @param configurations       a map of compiled configrations used for variable replacment
+   * @param keytab               the Keyab data for the relevant identity
+   * @param configurations       a map of compiled configurations used for variable replacement
+   * @param hostname             the hostname to use to replace _HOST in principal names, if necessary
+   * @param kerberosDetails      a KerberosDetails containing information about relevant Kerberos configuration
+   * @param updateJAASFile       true to update Ambari's JAAS file; false otherwise
    * @throws AmbariException
-   * @see ConfigureAmbariIdentitiesServerAction#installAmbariServerIdentity(String, String, String, ActionLog)
+   * @see ConfigureAmbariIdentitiesServerAction#configureJAAS(String, String, ActionLog)
    */
   private void installAmbariIdentity(KerberosIdentityDescriptor ambariServerIdentity,
-                                     Map<String, Map<String, String>> configurations) throws AmbariException {
+                                     Keytab keytab, Map<String, Map<String, String>> configurations,
+                                     String hostname,
+                                     KerberosDetails kerberosDetails,
+                                     boolean updateJAASFile) throws AmbariException {
     KerberosPrincipalDescriptor principalDescriptor = ambariServerIdentity.getPrincipalDescriptor();
+
     if (principalDescriptor != null) {
       String principal = variableReplacementHelper.replaceVariables(principalDescriptor.getValue(), configurations);
-      KerberosPrincipalEntity ambariServerPrincipalEntity = kerberosPrincipalDAO.find(principal);
 
-      if(ambariServerPrincipalEntity != null) {
-        KerberosKeytabDescriptor keytabDescriptor = ambariServerIdentity.getKeytabDescriptor();
-        if(keytabDescriptor != null) {
-          String keytabFilePath = variableReplacementHelper.replaceVariables(keytabDescriptor.getFile(), configurations);
+      // Replace _HOST with the supplied hostname is either exist
+      if (!StringUtils.isEmpty(hostname)) {
+        principal = principal.replace("_HOST", hostname);
+      }
 
-          injector.getInstance(ConfigureAmbariIdentitiesServerAction.class)
-              .installAmbariServerIdentity(principal, ambariServerPrincipalEntity.getCachedKeytabPath(), keytabFilePath, null);
+      KerberosKeytabDescriptor keytabDescriptor = ambariServerIdentity.getKeytabDescriptor();
+      if (keytabDescriptor != null) {
+        String destKeytabFilePath = variableReplacementHelper.replaceVariables(keytabDescriptor.getFile(), configurations);
+        File destKeytabFile = new File(destKeytabFilePath);
+
+        ConfigureAmbariIdentitiesServerAction configureAmbariIdentitiesServerAction = injector.getInstance(ConfigureAmbariIdentitiesServerAction.class);
+
+        if (keytab != null) {
+          try {
+            KerberosOperationHandler operationHandler = kerberosOperationHandlerFactory.getKerberosOperationHandler(kerberosDetails.getKdcType());
+            File tmpKeytabFile = createTemporaryFile();
+            try {
+              if ((operationHandler != null) && operationHandler.createKeytabFile(keytab, tmpKeytabFile)) {
+                String ownerName = variableReplacementHelper.replaceVariables(keytabDescriptor.getOwnerName(), configurations);
+                String ownerAccess = keytabDescriptor.getOwnerAccess();
+                boolean ownerWritable = "w".equalsIgnoreCase(ownerAccess) || "rw".equalsIgnoreCase(ownerAccess);
+                boolean ownerReadable = "r".equalsIgnoreCase(ownerAccess) || "rw".equalsIgnoreCase(ownerAccess);
+                String groupName = variableReplacementHelper.replaceVariables(keytabDescriptor.getGroupName(), configurations);
+                String groupAccess = keytabDescriptor.getGroupAccess();
+                boolean groupWritable = "w".equalsIgnoreCase(groupAccess) || "rw".equalsIgnoreCase(groupAccess);
+                boolean groupReadable = "r".equalsIgnoreCase(groupAccess) || "rw".equalsIgnoreCase(groupAccess);
+
+                configureAmbariIdentitiesServerAction.installAmbariServerIdentity(principal, tmpKeytabFile.getAbsolutePath(), destKeytabFilePath,
+                    ownerName, ownerReadable, ownerWritable, groupName, groupReadable, groupWritable, null);
+                LOG.debug("Successfully created keytab file for {} at {}", principal, destKeytabFile.getAbsolutePath());
+              } else {
+                LOG.error("Failed to create keytab file for {} at {}", principal, destKeytabFile.getAbsolutePath());
+              }
+            } finally {
+              tmpKeytabFile.delete();
+            }
+          } catch (KerberosOperationException e) {
+            throw new AmbariException(String.format("Failed to create keytab file for %s at %s: %s:",
+                principal, destKeytabFile.getAbsolutePath(), e.getLocalizedMessage()), e);
+          }
+        } else {
+          LOG.error("No keytab data is available to create the keytab file for {} at {}", principal, destKeytabFile.getAbsolutePath());
+        }
+
+        if (updateJAASFile) {
+          configureAmbariIdentitiesServerAction.configureJAAS(principal, destKeytabFile.getAbsolutePath(), null);
         }
       }
     }
@@ -1388,33 +1480,40 @@ public class KerberosHelperImpl implements KerberosHelper {
 
   /**
    * Creates the principal and cached keytab file for the specified identity, if it is determined to
-   * be user (or headless) identity
+   * be of the expected type - user (headless) or service.
    * <p/>
-   * If the identity is determined not to be a user identity, it is skipped.
+   * If the identity is not of the expected type, it will be skipped.
    *
    * @param identityDescriptor       the Kerberos identity to process
+   * @param expectedType             the expected principal type
    * @param kerberosEnvProperties    the kerberos-env properties
    * @param kerberosOperationHandler the relevant KerberosOperationHandler
    * @param configurations           the existing configurations for the cluster
-   * @return true if the identity was created; otherwise false
+   * @param hostname                 the hostname of the host to create the identity for (nullable)
+   * @return the relevant keytab data, if successful; otherwise null
    * @throws AmbariException
    */
-  private boolean createUserIdentity(KerberosIdentityDescriptor identityDescriptor,
-                                     Map<String, String> kerberosEnvProperties,
-                                     KerberosOperationHandler kerberosOperationHandler,
-                                     Map<String, Map<String, String>> configurations)
+  private Keytab createIdentity(KerberosIdentityDescriptor identityDescriptor,
+                                KerberosPrincipalType expectedType, Map<String, String> kerberosEnvProperties,
+                                KerberosOperationHandler kerberosOperationHandler,
+                                Map<String, Map<String, String>> configurations, String hostname)
       throws AmbariException {
 
-    boolean created = false;
+    Keytab keytab = null;
 
     if (identityDescriptor != null) {
       KerberosPrincipalDescriptor principalDescriptor = identityDescriptor.getPrincipalDescriptor();
 
       if (principalDescriptor != null) {
-        // If this principal indicates it is a user principal, continue, else skip it.
-        if (KerberosPrincipalType.USER == principalDescriptor.getType()) {
+        // If this principal type is expected, continue, else skip it.
+        if (expectedType == principalDescriptor.getType()) {
           String principal = variableReplacementHelper.replaceVariables(principalDescriptor.getValue(), configurations);
 
+          // Replace _HOST with the supplied hostname is either exist
+          if (!StringUtils.isEmpty(hostname)) {
+            principal = principal.replace("_HOST", hostname);
+          }
+
           // If this principal is already in the Ambari database, then don't try to recreate it or it's
           // keytab file.
           if (!kerberosPrincipalDAO.exists(principal)) {
@@ -1422,7 +1521,7 @@ public class KerberosHelperImpl implements KerberosHelper {
 
             result = injector.getInstance(CreatePrincipalsServerAction.class).createPrincipal(
                 principal,
-                false,
+                KerberosPrincipalType.SERVICE.equals(expectedType),
                 kerberosEnvProperties,
                 kerberosOperationHandler,
                 false,
@@ -1434,7 +1533,7 @@ public class KerberosHelperImpl implements KerberosHelper {
               KerberosKeytabDescriptor keytabDescriptor = identityDescriptor.getKeytabDescriptor();
 
               if (keytabDescriptor != null) {
-                Keytab keytab = injector.getInstance(CreateKeytabFilesServerAction.class).createKeytab(
+                keytab = injector.getInstance(CreateKeytabFilesServerAction.class).createKeytab(
                     principal,
                     result.getPassword(),
                     result.getKeyNumber(),
@@ -1447,15 +1546,13 @@ public class KerberosHelperImpl implements KerberosHelper {
                   throw new AmbariException("Failed to create the keytab for " + principal);
                 }
               }
-
-              created = true;
             }
           }
         }
       }
     }
 
-    return created;
+    return keytab;
   }
 
   /**
@@ -1657,7 +1754,7 @@ public class KerberosHelperImpl implements KerberosHelper {
 
     // Add the finalize stage...
     handler.addFinalizeOperationStage(cluster, clusterHostInfoJson, hostParamsJson, event,
-        dataDirectory, roleCommandOrder, requestStageContainer);
+        dataDirectory, roleCommandOrder, requestStageContainer, kerberosDetails);
 
     // If all goes well, set the appropriate states on the relevant ServiceComponentHosts
     for (ServiceComponentHost sch : schToProcess) {
@@ -1859,7 +1956,7 @@ public class KerberosHelperImpl implements KerberosHelper {
 
 
       handler.addFinalizeOperationStage(cluster, clusterHostInfoJson, hostParamsJson, event,
-          dataDirectory, roleCommandOrder, requestStageContainer);
+          dataDirectory, roleCommandOrder, requestStageContainer, kerberosDetails);
     }
 
     return requestStageContainer;
@@ -1954,24 +2051,16 @@ public class KerberosHelperImpl implements KerberosHelper {
    * @return a File pointing to the new temporary directory, or null if one was not created
    * @throws AmbariException if a new temporary directory cannot be created
    */
-  private File createTemporaryDirectory() throws AmbariException {
-    String tempDirectoryPath = configuration.getProperty(Configuration.SERVER_TMP_DIR.getKey());
-
-    if ((tempDirectoryPath == null) || tempDirectoryPath.isEmpty()) {
-      tempDirectoryPath = System.getProperty("java.io.tmpdir");
-    }
-
+  protected File createTemporaryDirectory() throws AmbariException {
     try {
-      if (tempDirectoryPath == null) {
-        throw new IOException("The System property 'java.io.tmpdir' does not specify a temporary directory");
-      }
+      File temporaryDirectory = getConfiguredTemporaryDirectory();
 
       File directory;
       int tries = 0;
       long now = System.currentTimeMillis();
 
       do {
-        directory = new File(tempDirectoryPath, String.format("%s%d-%d.d",
+        directory = new File(temporaryDirectory, String.format("%s%d-%d.d",
             KerberosServerAction.DATA_DIRECTORY_PREFIX, now, tries));
 
         if ((directory.exists()) || !directory.mkdirs()) {
@@ -1982,7 +2071,7 @@ public class KerberosHelperImpl implements KerberosHelper {
       } while ((directory == null) && (++tries < 100));
 
       if (directory == null) {
-        throw new IOException(String.format("Failed to create a temporary directory in %s", tempDirectoryPath));
+        throw new IOException(String.format("Failed to create a temporary directory in %s", temporaryDirectory));
       }
 
       return directory;
@@ -2085,6 +2174,43 @@ public class KerberosHelperImpl implements KerberosHelper {
     }
   }
 
+  /**
+   * Creates a temporary file within the system temporary directory
+   * <p/>
+   * The resulting file is to be removed by the caller when desired.
+   *
+   * @return a File pointing to the new temporary file, or null if one was not created
+   * @throws AmbariException if a new temporary directory cannot be created
+   */
+  protected File createTemporaryFile() throws AmbariException {
+    try {
+      return File.createTempFile("tmp", ".tmp", getConfiguredTemporaryDirectory());
+    } catch (IOException e) {
+      String message = "Failed to create a temporary file.";
+      LOG.error(message, e);
+      throw new AmbariException(message, e);
+    }
+  }
+
+  /**
+   * Gets the configured temporary directory.
+   *
+   * @return a File pointing to the configured temporary directory
+   * @throws IOException
+   */
+  protected File getConfiguredTemporaryDirectory() throws IOException {
+    String tempDirectoryPath = configuration.getServerTempDir();
+
+    if (StringUtils.isEmpty(tempDirectoryPath)) {
+      tempDirectoryPath = System.getProperty("java.io.tmpdir");
+    }
+
+    if (tempDirectoryPath == null) {
+      throw new IOException("The System property 'java.io.tmpdir' does not specify a temporary directory");
+    }
+
+    return new File(tempDirectoryPath);
+  }
 
   /**
    * Creates a new stage
@@ -2277,9 +2403,10 @@ public class KerberosHelperImpl implements KerberosHelper {
    *                           services
    * @param componentName      the name of a component for which to find results, null indicates all
    *                           components
-   * @param kerberosDescriptor the relevant Kerberos Descriptor     @return a list of KerberosIdentityDescriptors representing the active identities for the
-   * requested service component
+   * @param kerberosDescriptor the relevant Kerberos Descriptor
+   *                           requested service component
    * @param filterContext      the context to use for filtering identities based on the state of the cluster
+   * @return a list of KerberosIdentityDescriptors representing the active identities for the
    * @throws AmbariException if an error occurs processing the cluster's active identities
    */
   private List<KerberosIdentityDescriptor> getActiveIdentities(Cluster cluster,
@@ -2973,12 +3100,14 @@ public class KerberosHelperImpl implements KerberosHelper {
 
     public void addFinalizeOperationStage(Cluster cluster, String clusterHostInfoJson,
                                           String hostParamsJson, ServiceComponentHostServerActionEvent event,
-                                          File dataDirectory,
-                                          RoleCommandOrder roleCommandOrder, RequestStageContainer requestStageContainer)
+                                          File dataDirectory, RoleCommandOrder roleCommandOrder,
+                                          RequestStageContainer requestStageContainer,
+                                          KerberosDetails kerberosDetails)
         throws AmbariException {
 
       // Add the finalize stage...
       Map<String, String> commandParameters = new HashMap<String, String>();
+      commandParameters.put(KerberosServerAction.DEFAULT_REALM, kerberosDetails.getDefaultRealm());
       commandParameters.put(KerberosServerAction.AUTHENTICATED_USER_NAME, ambariManagementController.getAuthName());
       if (dataDirectory != null) {
         commandParameters.put(KerberosServerAction.DATA_DIRECTORY, dataDirectory.getAbsolutePath());

http://git-wip-us.apache.org/repos/asf/ambari/blob/1bf20690/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 10647e8..80acd0d 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
@@ -126,7 +126,16 @@ public class ConfigureAmbariIdentitiesServerAction extends KerberosServerAction
           File srcKeytabFile = new File(hostDirectory, DigestUtils.sha1Hex(destKeytabFilePath));
 
           if (srcKeytabFile.exists()) {
-            installAmbariServerIdentity(evaluatedPrincipal, srcKeytabFile.getAbsolutePath(), destKeytabFilePath, actionLog);
+            String ownerAccess = identityRecord.get(KerberosIdentityDataFileReader.KEYTAB_FILE_OWNER_ACCESS);
+            boolean ownerWritable = "w".equalsIgnoreCase(ownerAccess) || "rw".equalsIgnoreCase(ownerAccess);
+            boolean ownerReadable = "r".equalsIgnoreCase(ownerAccess) || "rw".equalsIgnoreCase(ownerAccess);
+            String groupAccess = identityRecord.get(KerberosIdentityDataFileReader.KEYTAB_FILE_OWNER_ACCESS);
+            boolean groupWritable = "w".equalsIgnoreCase(groupAccess) || "rw".equalsIgnoreCase(groupAccess);
+            boolean groupReadable = "r".equalsIgnoreCase(groupAccess) || "rw".equalsIgnoreCase(groupAccess);
+
+            installAmbariServerIdentity(evaluatedPrincipal, srcKeytabFile.getAbsolutePath(), destKeytabFilePath,
+                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))) {
               // Create/update the JAASFile...
@@ -147,6 +156,12 @@ public class ConfigureAmbariIdentitiesServerAction extends KerberosServerAction
    * @param principal          the ambari server principal name
    * @param srcKeytabFilePath  the source location of the ambari server keytab file
    * @param destKeytabFilePath the destination location of the ambari server keytab file
+   * @param ownerName          the username for the owner of the generated keytab file
+   * @param ownerReadable      true if the owner should be able to read this file; otherwise false
+   * @param ownerWritable      true if the owner should be able to write to this file; otherwise false
+   * @param groupName          the name of the group for the generated keytab file
+   * @param groupReadable      true if the group should be able to read this file; otherwise false
+   * @param groupWritable      true if the group should be able to write to this file; otherwise false
    * @param actionLog          the logger
    * @return true if success; false otherwise
    * @throws AmbariException
@@ -154,46 +169,38 @@ public class ConfigureAmbariIdentitiesServerAction extends KerberosServerAction
   public boolean installAmbariServerIdentity(String principal,
                                              String srcKeytabFilePath,
                                              String destKeytabFilePath,
+                                             String ownerName, boolean ownerReadable, boolean ownerWritable,
+                                             String groupName, boolean groupReadable, boolean groupWritable,
                                              ActionLog actionLog) throws AmbariException {
 
-    // Use sudo to copy the file into place....
     try {
-      ShellCommandUtil.Result result;
+      // Copy the keytab file into place (creating the parent directory, if necessary...
+      copyFile(srcKeytabFilePath, destKeytabFilePath);
+      setFileACL(destKeytabFilePath,
+          ownerName, ownerReadable, ownerWritable,
+          groupName, groupReadable, groupWritable);
 
-      // Ensure the parent directory exists...
-      File destKeytabFile = new File(destKeytabFilePath);
-      result = ShellCommandUtil.mkdir(destKeytabFile.getParent(), true);
-      if (!result.isSuccessful()) {
-        throw new AmbariException(result.getStderr());
-      }
+      String ambariServerHostName = StageUtils.getHostName();
+      HostEntity ambariServerHostEntity = hostDAO.findByName(ambariServerHostName);
+      Long ambariServerHostID = (ambariServerHostEntity == null)
+          ? null
+          : ambariServerHostEntity.getHostId();
 
-      // Copy the keytab file into place...
-      result = ShellCommandUtil.copyFile(srcKeytabFilePath, destKeytabFilePath, true, true);
-      if (!result.isSuccessful()) {
-        throw new AmbariException(result.getStderr());
-      } else {
-        String ambariServerHostName = StageUtils.getHostName();
-        HostEntity ambariServerHostEntity = hostDAO.findByName(ambariServerHostName);
-        Long ambariServerHostID = (ambariServerHostEntity == null)
-            ? null
-            : ambariServerHostEntity.getHostId();
-
-        if (ambariServerHostID == null) {
-          String message = String.format("Failed to add the kerberos_principal_host record for %s on " +
-                  "the Ambari server host since the host id for Ambari server host, %s, was not found." +
-                  "  This is not an error if an Ambari agent is not installed on the Ambari server host.",
-              principal, ambariServerHostName);
-          LOG.warn(message);
-          if(actionLog != null) {
-            actionLog.writeStdErr(message);
-          }
-        } else if (!kerberosPrincipalHostDAO.exists(principal, ambariServerHostID)) {
-          kerberosPrincipalHostDAO.create(principal, ambariServerHostID);
+      if (ambariServerHostID == null) {
+        String message = String.format("Failed to add the kerberos_principal_host record for %s on " +
+                "the Ambari server host since the host id for Ambari server host, %s, was not found." +
+                "  This is not an error if an Ambari agent is not installed on the Ambari server host.",
+            principal, ambariServerHostName);
+        LOG.warn(message);
+        if (actionLog != null) {
+          actionLog.writeStdErr(message);
         }
+      } else if (!kerberosPrincipalHostDAO.exists(principal, ambariServerHostID)) {
+        kerberosPrincipalHostDAO.create(principal, ambariServerHostID);
+      }
 
-        if(actionLog != null) {
-          actionLog.writeStdOut(String.format("Created Ambari server keytab file for %s at %s", principal, destKeytabFile));
-        }
+      if (actionLog != null) {
+        actionLog.writeStdOut(String.format("Created Ambari server keytab file for %s at %s", principal, destKeytabFilePath));
       }
     } catch (InterruptedException | IOException e) {
       throw new AmbariException(e.getLocalizedMessage(), e);
@@ -202,8 +209,16 @@ public class ConfigureAmbariIdentitiesServerAction extends KerberosServerAction
     return true;
   }
 
-  private void configureJAAS(String evaluatedPrincipal, String keytabFilePath, ActionLog actionLog) {
-    String jaasConfPath = System.getProperty(KerberosChecker.JAVA_SECURITY_AUTH_LOGIN_CONFIG);
+  /**
+   * Configure Ambari's JAAS file to reflect the principal name and keytab file for Ambari's Kerberos
+   * identity.
+   *
+   * @param principal      the Ambari server's principal name
+   * @param keytabFilePath the absolute path to the Ambari server's keytab file
+   * @param actionLog      the logger
+   */
+  public void configureJAAS(String principal, String keytabFilePath, ActionLog actionLog) {
+    String jaasConfPath = getJAASConfFilePath();
     if (jaasConfPath != null) {
       File jaasConfigFile = new File(jaasConfPath);
       try {
@@ -211,16 +226,16 @@ public class ConfigureAmbariIdentitiesServerAction extends KerberosServerAction
         File oldJaasConfigFile = new File(jaasConfPath + ".bak");
         FileUtils.writeStringToFile(oldJaasConfigFile, jaasConfig);
         jaasConfig = jaasConfig.replaceFirst(KEYTAB_PATTERN, "keyTab=\"" + keytabFilePath + "\"");
-        jaasConfig = jaasConfig.replaceFirst(PRINCIPAL_PATTERN, "principal=\"" + evaluatedPrincipal + "\"");
+        jaasConfig = jaasConfig.replaceFirst(PRINCIPAL_PATTERN, "principal=\"" + principal + "\"");
         FileUtils.writeStringToFile(jaasConfigFile, jaasConfig);
-        String message = String.format("JAAS config file %s modified successfully for principal %s.", jaasConfigFile
-            .getName(), evaluatedPrincipal);
+        String message = String.format("JAAS config file %s modified successfully for principal %s.",
+            jaasConfigFile.getName(), principal);
         if (actionLog != null) {
           actionLog.writeStdOut(message);
         }
       } catch (IOException e) {
-        String message = String.format("Failed to configure JAAS file %s for %s - %s", jaasConfigFile,
-            evaluatedPrincipal, e.getMessage());
+        String message = String.format("Failed to configure JAAS file %s for %s - %s",
+            jaasConfigFile, principal, e.getMessage());
         if (actionLog != null) {
           actionLog.writeStdErr(message);
         }
@@ -236,4 +251,97 @@ public class ConfigureAmbariIdentitiesServerAction extends KerberosServerAction
     }
   }
 
+  /**
+   * Copies the specified source file to the specified destination path, creating any needed parent
+   * directories.
+   * <p>
+   * This method is mocked in unit tests to avoid dealing with ShellCommandUtil in a mocked env.
+   *
+   * @param srcKeytabFilePath  the source location of the ambari server keytab file
+   * @param destKeytabFilePath the destination location of the ambari server keytab file
+   * @throws IOException
+   * @throws InterruptedException
+   * @throws AmbariException
+   * @see ShellCommandUtil#mkdir(String, boolean);
+   * @see ShellCommandUtil#copyFile(String, String, boolean, boolean)
+   */
+  void copyFile(String srcKeytabFilePath, String destKeytabFilePath)
+      throws IOException, InterruptedException {
+
+    ShellCommandUtil.Result result;
+
+    // Create the parent directory if necessary (using sudo)
+    File destKeytabFile = new File(destKeytabFilePath);
+    result = ShellCommandUtil.mkdir(destKeytabFile.getParent(), true);
+    if (!result.isSuccessful()) {
+      throw new AmbariException(result.getStderr());
+    }
+
+    // Copy the file (using sudo)
+    result = ShellCommandUtil.copyFile(srcKeytabFilePath, destKeytabFilePath, true, true);
+    if (!result.isSuccessful()) {
+      throw new AmbariException(result.getStderr());
+    }
+  }
+
+  /**
+   * Sets the access control list for this specified file.
+   * <p>
+   * The owner and group for the file is set as well as the owner's and group's ability to read and write
+   * the file.
+   * <p>
+   * The result of the operation to set the group for the file is ignored since it is possible that
+   * the group does not exist when performing this operation. It is expected this issue will be remedied
+   * when the group becomes available.
+   * <p>
+   * Access for other users is denied and the file is assumed to not be executeable by anyone.
+   *
+   * @param filePath      the path to the file
+   * @param ownerName     the username for the owner of the generated keytab file
+   * @param ownerWritable true if the owner should be able to write to this file; otherwise false
+   * @param ownerReadable true if the owner should be able to read this file; otherwise false
+   * @param groupName     the name of the group for the generated keytab file
+   * @param groupWritable true if the group should be able to write to this file; otherwise false
+   * @param groupReadable true if the group should be able to read this file; otherwise false
+   * @throws AmbariException if an error occurs setting the permissions on the fils
+   */
+  void setFileACL(String filePath,
+                          String ownerName, boolean ownerReadable, boolean ownerWritable,
+                          String groupName, boolean groupReadable, boolean groupWritable)
+      throws AmbariException {
+
+    ShellCommandUtil.Result result;
+
+    result = ShellCommandUtil.setFileOwner(filePath, ownerName);
+
+    if (result.isSuccessful()) {
+      result = ShellCommandUtil.setFileGroup(filePath, groupName);
+
+      if (!result.isSuccessful()) {
+        // Ignore, but log, this it is possible that the group does not exist when performing this operation
+        LOG.warn("Failed to set the group for the file at {} to {}: {}", filePath, groupName, result.getStderr());
+      }
+
+      result = ShellCommandUtil.setFileMode(filePath,
+          ownerReadable, ownerWritable, false,
+          groupReadable, groupWritable, false,
+          false, false, false);
+    }
+
+    if (!result.isSuccessful()) {
+      throw new AmbariException(result.getStderr());
+    }
+  }
+
+  /**
+   * Gets the location of Ambari's JAAS config file.
+   * <p>
+   * This method is mocked in unit tests to avoid having to alter the System properties in
+   * order to locate the test JAAS config file.
+   *
+   * @return the path to Ambari's JAAS config file
+   */
+  String getJAASConfFilePath() {
+    return System.getProperty(KerberosChecker.JAVA_SECURITY_AUTH_LOGIN_CONFIG);
+  }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/1bf20690/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/FinalizeKerberosServerAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/FinalizeKerberosServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/FinalizeKerberosServerAction.java
index 7205d09..d485eb3 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/FinalizeKerberosServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/FinalizeKerberosServerAction.java
@@ -26,14 +26,19 @@ import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Host;
 import org.apache.ambari.server.state.SecurityState;
 import org.apache.ambari.server.state.ServiceComponentHost;
+import org.apache.ambari.server.utils.ShellCommandUtil;
+import org.apache.ambari.server.utils.StageUtils;
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
 public class FinalizeKerberosServerAction extends KerberosServerAction {
@@ -42,7 +47,10 @@ public class FinalizeKerberosServerAction extends KerberosServerAction {
   /**
    * Processes an identity as necessary.
    * <p/>
-   * This method is not used since the {@link #processIdentities(java.util.Map)} is not invoked
+   * This implementation ensures that keytab files for the Ambari identities have the correct
+   * permissions.  This is important in the event a secure cluster was created via Blueprints since
+   * some user accounts and groups may not have been available (at the OS level) when the keytab files
+   * were created.
    *
    * @param identityRecord           a Map containing the data for the current identity record
    * @param evaluatedPrincipal       a String indicating the relevant principal
@@ -61,6 +69,91 @@ public class FinalizeKerberosServerAction extends KerberosServerAction {
                                           Map<String, Object> requestSharedDataContext)
       throws AmbariException {
 
+    if (identityRecord != null) {
+      // If the record's HOSTNAME value is "ambari-server", rather than an actual hostname it will
+      // not match the Ambari server's host name. This will occur if the there is no agent installed
+      // on the Ambari server host.  This is ok, since any keytab files installed on the Ambari server
+      // host will already have the permissions set so that only the Ambari server can read it.
+      // There is no need to update the permissions for those keytab files so that installed services
+      // can access them since no services will be installed on the host.
+      if (StageUtils.getHostName().equals(identityRecord.get(KerberosIdentityDataFile.HOSTNAME))) {
+
+        // If the principal name exists in one of the shared data maps, it has been processed by the
+        // current "Enable Kerberos" or "Add component" workflow and therefore should already have
+        // the correct permissions assigned. The relevant keytab files can be skipped.
+        Map<String, String> principalPasswordMap = getPrincipalPasswordMap(requestSharedDataContext);
+        if ((principalPasswordMap == null) || !principalPasswordMap.containsKey(evaluatedPrincipal)) {
+
+          String keytabFilePath = identityRecord.get(KerberosIdentityDataFile.KEYTAB_FILE_PATH);
+
+          if (!StringUtils.isEmpty(keytabFilePath)) {
+            Set<String> visited = (Set<String>) requestSharedDataContext.get(this.getClass().getName() + "_visited");
+
+            if (!visited.contains(keytabFilePath)) {
+              String ownerName = identityRecord.get(KerberosIdentityDataFile.KEYTAB_FILE_OWNER_NAME);
+              String ownerAccess = identityRecord.get(KerberosIdentityDataFileReader.KEYTAB_FILE_OWNER_ACCESS);
+              boolean ownerWritable = "w".equalsIgnoreCase(ownerAccess) || "rw".equalsIgnoreCase(ownerAccess);
+              boolean ownerReadable = "r".equalsIgnoreCase(ownerAccess) || "rw".equalsIgnoreCase(ownerAccess);
+              String groupName = identityRecord.get(KerberosIdentityDataFile.KEYTAB_FILE_GROUP_NAME);
+              String groupAccess = identityRecord.get(KerberosIdentityDataFileReader.KEYTAB_FILE_OWNER_ACCESS);
+              boolean groupWritable = "w".equalsIgnoreCase(groupAccess) || "rw".equalsIgnoreCase(groupAccess);
+              boolean groupReadable = "r".equalsIgnoreCase(groupAccess) || "rw".equalsIgnoreCase(groupAccess);
+
+              ShellCommandUtil.Result result;
+              String message;
+
+              result = ShellCommandUtil.setFileOwner(keytabFilePath, ownerName);
+              if (result.isSuccessful()) {
+                message = String.format("Updated the owner of the keytab file at %s to %s",
+                    keytabFilePath, ownerName);
+                LOG.info(message);
+                actionLog.writeStdOut(message);
+              } else {
+                message = String.format("Failed to update the owner of the keytab file at %s to %s: %s",
+                    keytabFilePath, ownerName, result.getStderr());
+                LOG.error(message);
+                actionLog.writeStdOut(message);
+                actionLog.writeStdErr(message);
+              }
+
+              result = ShellCommandUtil.setFileGroup(keytabFilePath, groupName);
+              if (result.isSuccessful()) {
+                message = String.format("Updated the group of the keytab file at %s to %s",
+                    keytabFilePath, groupName);
+                LOG.info(message);
+                actionLog.writeStdOut(message);
+              } else {
+                message = String.format("Failed to update the group of the keytab file at %s to %s: %s",
+                    keytabFilePath, groupName, result.getStderr());
+                LOG.error(message);
+                actionLog.writeStdOut(message);
+                actionLog.writeStdErr(message);
+              }
+
+              result = ShellCommandUtil.setFileMode(keytabFilePath,
+                  ownerReadable, ownerWritable, false,
+                  groupReadable, groupWritable, false,
+                  false, false, false);
+              if (result.isSuccessful()) {
+                message = String.format("Updated the access mode of the keytab file at %s to owner:'%s' and group:'%s'",
+                    keytabFilePath, ownerAccess, groupAccess);
+                LOG.info(message);
+                actionLog.writeStdOut(message);
+              } else {
+                message = String.format("Failed to update the access mode of the keytab file at %s to owner:'%s' and group:'%s': %s",
+                    keytabFilePath, ownerAccess, groupAccess, result.getStderr());
+                LOG.error(message);
+                actionLog.writeStdOut(message);
+                actionLog.writeStdErr(message);
+              }
+
+              visited.add(keytabFilePath);
+            }
+          }
+        }
+      }
+    }
+
     return null;
   }
 
@@ -93,20 +186,27 @@ public class FinalizeKerberosServerAction extends KerberosServerAction {
 
             sch.setSecurityState(sch.getDesiredSecurityState());
             ChangeSecurityStateKerberosAuditEvent auditEvent = ChangeSecurityStateKerberosAuditEvent.builder()
-              .withTimestamp(System.currentTimeMillis())
-              .withService(sch.getServiceName())
-              .withComponent(sch.getServiceComponentName())
-              .withHostName(sch.getHostName())
-              .withState(sch.getDesiredSecurityState().toString())
-              .withRequestId(getHostRoleCommand().getRequestId())
-              .withTaskId(getHostRoleCommand().getTaskId())
-              .build();
+                .withTimestamp(System.currentTimeMillis())
+                .withService(sch.getServiceName())
+                .withComponent(sch.getServiceComponentName())
+                .withHostName(sch.getHostName())
+                .withState(sch.getDesiredSecurityState().toString())
+                .withRequestId(getHostRoleCommand().getRequestId())
+                .withTaskId(getHostRoleCommand().getTaskId())
+                .build();
             auditLog(auditEvent);
           }
         }
       }
     }
 
+    // Ensure the keytab files for the Ambari identities have the correct permissions
+    // This is important in the event a secure cluster was created via Blueprints since some
+    // user accounts and group may not have been created when the keytab files were created.
+    requestSharedDataContext.put(this.getClass().getName() + "_visited", new HashSet<String>());
+    processIdentities(requestSharedDataContext);
+    requestSharedDataContext.remove(this.getClass().getName() + "_visited");
+
     // Make sure this is a relevant directory. We don't want to accidentally allow _ANY_ directory
     // to be deleted.
     if ((dataDirectoryPath != null) && dataDirectoryPath.contains("/" + DATA_DIRECTORY_PREFIX)) {

http://git-wip-us.apache.org/repos/asf/ambari/blob/1bf20690/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/KerberosOperationHandler.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/KerberosOperationHandler.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/KerberosOperationHandler.java
index 02cbb57..51d1398 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/KerberosOperationHandler.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/KerberosOperationHandler.java
@@ -409,7 +409,7 @@ public abstract class KerberosOperationHandler {
    * @return true if the keytab file was successfully created; false otherwise
    * @throws KerberosOperationException
    */
-  protected boolean createKeytabFile(Keytab keytab, File destinationKeytabFile)
+  public boolean createKeytabFile(Keytab keytab, File destinationKeytabFile)
       throws KerberosOperationException {
 
     if (destinationKeytabFile == null) {

http://git-wip-us.apache.org/repos/asf/ambari/blob/1bf20690/ambari-server/src/main/java/org/apache/ambari/server/utils/ShellCommandUtil.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/utils/ShellCommandUtil.java b/ambari-server/src/main/java/org/apache/ambari/server/utils/ShellCommandUtil.java
index 39981ef..8aa9c08 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/utils/ShellCommandUtil.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/utils/ShellCommandUtil.java
@@ -17,6 +17,7 @@
  */
 package org.apache.ambari.server.utils;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -39,6 +40,16 @@ public class ShellCommandUtil {
   private static final String KEY_TOKEN = "-key ";
   private static final String AMBARI_SUDO = "ambari-sudo.sh";
 
+  private static final int MODE_OWNER_READABLE = 400;
+  private static final int MODE_OWNER_WRITABLE = 200;
+  private static final int MODE_OWNER_EXECUTABLE = 100;
+  private static final int MODE_GROUP_READABLE = 40;
+  private static final int MODE_GROUP_WRITABLE = 20;
+  private static final int MODE_GROUP_EXECUTABLE = 10;
+  private static final int MODE_OTHER_READABLE = 4;
+  private static final int MODE_OTHER_WRITABLE = 2;
+  private static final int MODE_OTHER_EXECUTABLE = 1;
+
   /*
   public static String LogAndReturnOpenSslExitCode(String command, int exitCode) {
     logOpenSslExitCode(command, exitCode);
@@ -181,6 +192,116 @@ public class ShellCommandUtil {
   }
 
   /**
+   * Sets the owner for a file.
+   *
+   * @param path      the path to the file
+   * @param ownerName the owner's local username
+   * @return the result of the operation
+   */
+  public static Result setFileOwner(String path, String ownerName) {
+    if (LINUX) {
+      // Set the file owner, if the owner's username is given
+      if (!StringUtils.isEmpty(ownerName)) {
+        try {
+          return runCommand(new String[]{"chown", ownerName, path}, null, null, true);
+        } catch (IOException e) {
+          // Improbable
+          LOG.warn(String.format("Can not perform chown %s %s", ownerName, path), e);
+          return new Result(-1, "", "Cannot perform operation: " + e.getLocalizedMessage());
+        } catch (InterruptedException e) {
+          LOG.warn(String.format("Can not perform chown %s %s", ownerName, path), e);
+          return new Result(-1, "", "Cannot perform operation: " + e.getLocalizedMessage());
+        }
+      } else {
+        return new Result(0, "", "");
+      }
+    } else {
+      LOG.debug(String.format("Not performing chown command for file %s " +
+          "because current OS is not Linux ", path));
+      return new Result(-1, "", "Cannot perform operation: The current OS is not Linux");
+    }
+  }
+
+  /**
+   * Sets the group for a file.
+   *
+   * @param path      the path to the file
+   * @param groupName the group name
+   * @return the result of the operation
+   */
+  public static Result setFileGroup(String path,  String groupName) {
+    if (LINUX) {
+      // Set the file's group, if the group name is given
+      if (!StringUtils.isEmpty(groupName)) {
+        try {
+          return runCommand(new String[]{"chgrp", groupName, path}, null, null, true);
+        } catch (IOException e) {
+          // Improbable
+          LOG.warn(String.format("Can not perform chgrp %s %s", groupName, path), e);
+          return new Result(-1, "", "Cannot perform operation: " + e.getLocalizedMessage());
+        } catch (InterruptedException e) {
+          LOG.warn(String.format("Can not perform chgrp %s %s", groupName, path), e);
+          return new Result(-1, "", "Cannot perform operation: " + e.getLocalizedMessage());
+        }
+      } else {
+        return new Result(0, "", "");
+      }
+    } else {
+      LOG.debug(String.format("Not performing chgrp command for file %s " +
+          "because current OS is not Linux ", path));
+      return new Result(-1, "", "Cannot perform operation: The current OS is not Linux");
+    }
+  }
+
+  /**
+   * Set the access modes for a file
+   *
+   * @param path            the path to the file
+   * @param ownerWritable   true if the owner should be able to write to this file; otherwise false
+   * @param ownerReadable   true if the owner should be able to read this file; otherwise false
+   * @param ownerExecutable true if the owner should be able to execute this file; otherwise false
+   * @param groupWritable   true if the group should be able to write to this file; otherwise false
+   * @param groupReadable   true if the group should be able to read this file; otherwise false
+   * @param groupExecutable true if the group should be able to execute this file; otherwise false
+   * @param otherReadable   true if other users should be able to read this file; otherwise false
+   * @param otherWritable   true if other users should be able to write to this file; otherwise false
+   * @param otherExecutable true if other users should be able to execute this file; otherwise false
+   * @return the result of the operation
+   */
+  public static Result setFileMode(String path,
+                                   boolean ownerReadable, boolean ownerWritable, boolean ownerExecutable,
+                                   boolean groupReadable, boolean groupWritable, boolean groupExecutable,
+                                   boolean otherReadable, boolean otherWritable, boolean otherExecutable) {
+    if (LINUX) {
+      int modeValue = ((ownerReadable) ? MODE_OWNER_READABLE : 0) +
+          ((ownerWritable) ? MODE_OWNER_WRITABLE : 0) +
+          ((ownerExecutable) ? MODE_OWNER_EXECUTABLE : 0) +
+          ((groupReadable) ? MODE_GROUP_READABLE : 0) +
+          ((groupWritable) ? MODE_GROUP_WRITABLE : 0) +
+          ((groupExecutable) ? MODE_GROUP_EXECUTABLE : 0) +
+          ((otherReadable) ? MODE_OTHER_READABLE : 0) +
+          ((otherWritable) ? MODE_OTHER_WRITABLE : 0) +
+          ((otherExecutable) ? MODE_OTHER_EXECUTABLE : 0);
+      String mode = String.format("%04d", modeValue);
+
+      try {
+        return runCommand(new String[]{"chmod", mode, path}, null, null, true);
+      } catch (IOException e) {
+        // Improbable
+        LOG.warn(String.format("Can not perform chmod %s %s", mode, path), e);
+        return new Result(-1, "", "Cannot perform operation: " + e.getLocalizedMessage());
+      } catch (InterruptedException e) {
+        LOG.warn(String.format("Can not perform chmod %s %s", mode, path), e);
+        return new Result(-1, "", "Cannot perform operation: " + e.getLocalizedMessage());
+      }
+    } else {
+      LOG.debug(String.format("Not performing chmod command for file %s " +
+          "because current OS is not Linux ", path));
+      return new Result(-1, "", "Cannot perform operation: The current OS is not Linux");
+    }
+  }
+
+  /**
    * Test if a file or directory exists
    *
    * @param path the path to test

http://git-wip-us.apache.org/repos/asf/ambari/blob/1bf20690/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 3aad080..0644f57 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
@@ -9,7 +9,7 @@
       "name": "spnego",
       "principal": {
         "value": "HTTP/_HOST@${realm}",
-        "type" : "service"
+        "type": "service"
       },
       "keytab": {
         "file": "${keytab_dir}/spnego.service.keytab",
@@ -27,9 +27,9 @@
       "name": "smokeuser",
       "principal": {
         "value": "${cluster-env/smokeuser}-${cluster_name|toLower()}@${realm}",
-        "type" : "user",
+        "type": "user",
         "configuration": "cluster-env/smokeuser_principal_name",
-        "local_username" : "${cluster-env/smokeuser}"
+        "local_username": "${cluster-env/smokeuser}"
       },
       "keytab": {
         "file": "${keytab_dir}/smokeuser.headless.keytab",
@@ -48,13 +48,15 @@
       "name": "ambari-server",
       "principal": {
         "value": "ambari-server-${cluster_name|toLower()}@${realm}",
-        "type" : "user",
+        "type": "user",
         "configuration": "cluster-env/ambari_principal_name"
       },
       "keytab": {
-        "file": "${keytab_dir}/ambari.server.keytab"
+        "file": "${keytab_dir}/ambari.server.keytab",
+        "owner": {
+          "access": "r"
+        }
       }
     }
   ]
-
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/1bf20690/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 7e6a056..b2eb738 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
@@ -37,16 +37,19 @@ import org.apache.ambari.server.audit.AuditLogger;
 import org.apache.ambari.server.configuration.Configuration;
 import org.apache.ambari.server.controller.internal.RequestStageContainer;
 import org.apache.ambari.server.controller.spi.ClusterController;
+import org.apache.ambari.server.controller.utilities.KerberosChecker;
 import org.apache.ambari.server.metadata.RoleCommandOrder;
 import org.apache.ambari.server.orm.DBAccessor;
 import org.apache.ambari.server.orm.dao.ArtifactDAO;
 import org.apache.ambari.server.orm.dao.HostRoleCommandDAO;
+import org.apache.ambari.server.orm.dao.KerberosPrincipalDAO;
 import org.apache.ambari.server.security.SecurityHelper;
 import org.apache.ambari.server.security.credential.PrincipalKeyCredential;
 import org.apache.ambari.server.security.encryption.CredentialStoreService;
 import org.apache.ambari.server.security.encryption.CredentialStoreServiceImpl;
 import org.apache.ambari.server.security.encryption.CredentialStoreType;
 import org.apache.ambari.server.serveraction.ActionLog;
+import org.apache.ambari.server.serveraction.kerberos.ConfigureAmbariIdentitiesServerAction;
 import org.apache.ambari.server.serveraction.kerberos.CreateKeytabFilesServerAction;
 import org.apache.ambari.server.serveraction.kerberos.CreatePrincipalsServerAction;
 import org.apache.ambari.server.serveraction.kerberos.KDCType;
@@ -94,9 +97,13 @@ import org.easymock.EasyMockSupport;
 import org.easymock.IAnswer;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 
 import javax.persistence.EntityManager;
+import java.io.File;
+import java.lang.reflect.Method;
 import java.net.InetAddress;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
@@ -113,6 +120,7 @@ import java.util.concurrent.TimeUnit;
 
 import static org.easymock.EasyMock.anyLong;
 import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.anyString;
 import static org.easymock.EasyMock.capture;
 import static org.easymock.EasyMock.eq;
 import static org.easymock.EasyMock.expect;
@@ -139,6 +147,9 @@ public class KerberosHelperTest extends EasyMockSupport {
   private final TopologyManager topologyManager = createMock(TopologyManager.class);
   private final Configuration configuration = createMock(Configuration.class);
 
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
   @Before
   public void setUp() throws Exception {
     reset(clusterController);
@@ -183,9 +194,22 @@ public class KerberosHelperTest extends EasyMockSupport {
           public boolean removePrincipal(String principal) throws KerberosOperationException {
             return false;
           }
+
+          @Override
+          public boolean createKeytabFile(Keytab keytab, File destinationKeytabFile) throws KerberosOperationException {
+            return true;
+          }
         })
         .anyTimes();
 
+    Method methodGetConfiguredTemporaryDirectory = KerberosHelperImpl.class.getDeclaredMethod("getConfiguredTemporaryDirectory");
+
+    final KerberosHelperImpl kerberosHelper = createMockBuilder(KerberosHelperImpl.class)
+        .addMockedMethod(methodGetConfiguredTemporaryDirectory)
+        .createMock();
+
+    expect(kerberosHelper.getConfiguredTemporaryDirectory()).andReturn(temporaryFolder.getRoot()).anyTimes();
+
     injector = Guice.createInjector(new AbstractModule() {
 
       @Override
@@ -210,14 +234,18 @@ public class KerberosHelperTest extends EasyMockSupport {
         bind(KerberosDescriptorFactory.class).toInstance(kerberosDescriptorFactory);
         bind(KerberosConfigDataFileWriterFactory.class).toInstance(kerberosConfigDataFileWriterFactory);
         bind(StackManagerFactory.class).toInstance(createNiceMock(StackManagerFactory.class));
-        bind(KerberosHelper.class).to(KerberosHelperImpl.class);
+        bind(KerberosHelper.class).toInstance(kerberosHelper);
         bind(CredentialStoreService.class).to(CredentialStoreServiceImpl.class);
         bind(CreatePrincipalsServerAction.class).toInstance(createMock(CreatePrincipalsServerAction.class));
         bind(CreateKeytabFilesServerAction.class).toInstance(createMock(CreateKeytabFilesServerAction.class));
+        bind(ConfigureAmbariIdentitiesServerAction.class).toInstance(createMock(ConfigureAmbariIdentitiesServerAction.class));
         bind(StackAdvisorHelper.class).toInstance(createMock(StackAdvisorHelper.class));
         bind(HostRoleCommandDAO.class).toInstance(createNiceMock(HostRoleCommandDAO.class));
         bind(AuditLogger.class).toInstance(createNiceMock(AuditLogger.class));
         bind(ArtifactDAO.class).toInstance(createNiceMock(ArtifactDAO.class));
+        bind(KerberosPrincipalDAO.class).toInstance(createNiceMock(KerberosPrincipalDAO.class));
+
+        requestStaticInjection(KerberosChecker.class);
       }
     });
 
@@ -225,10 +253,11 @@ public class KerberosHelperTest extends EasyMockSupport {
     StageUtils.setTopologyManager(topologyManager);
     expect(topologyManager.getPendingHostComponents()).andReturn(
         Collections.<String, Collection<String>>emptyMap()).anyTimes();
-    
+
     StageUtils.setConfiguration(configuration);
     expect(configuration.getApiSSLAuthentication()).andReturn(false).anyTimes();
     expect(configuration.getClientApiPort()).andReturn(8080).anyTimes();
+    expect(configuration.getServerTempDir()).andReturn(temporaryFolder.getRoot().getAbsolutePath()).anyTimes();
 
     CredentialStoreService credentialStoreService = injector.getInstance(CredentialStoreService.class);
     if (!credentialStoreService.isInitialized(CredentialStoreType.TEMPORARY)) {
@@ -1007,7 +1036,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     final Config krb5ConfConfig = createMock(Config.class);
     expect(krb5ConfConfig.getProperties()).andReturn(krb5ConfProperties).anyTimes();
 
-    final Cluster cluster = createMockCluster(Collections.singleton(host), SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
+    final Cluster cluster = createMockCluster("c1", Collections.singleton(host), SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
     expect(cluster.getServices())
         .andReturn(new HashMap<String, Service>() {
           {
@@ -1198,7 +1227,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     final Config krb5ConfConfig = createMock(Config.class);
     expect(krb5ConfConfig.getProperties()).andReturn(krb5ConfProperties).anyTimes();
 
-    final Cluster cluster = createMockCluster(Collections.singleton(host), SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
+    final Cluster cluster = createMockCluster("c1", Collections.singleton(host), SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
     expect(cluster.getServices())
         .andReturn(new HashMap<String, Service>() {
           {
@@ -1384,7 +1413,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     final Config krb5ConfConfig = createMock(Config.class);
     expect(krb5ConfConfig.getProperties()).andReturn(krb5ConfProperties).anyTimes();
 
-    final Cluster cluster = createMockCluster(Collections.singleton(host), SecurityType.NONE, krb5ConfConfig, kerberosEnvConfig);
+    final Cluster cluster = createMockCluster("c1", Collections.singleton(host), SecurityType.NONE, krb5ConfConfig, kerberosEnvConfig);
     expect(cluster.getServices())
         .andReturn(new HashMap<String, Service>() {
           {
@@ -1580,7 +1609,7 @@ public class KerberosHelperTest extends EasyMockSupport {
         ? Arrays.asList(host, hostInvalid)
         : Collections.singleton(host);
 
-    final Cluster cluster = createMockCluster(hosts, SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
+    final Cluster cluster = createMockCluster("c1", hosts, SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
     expect(cluster.getServices())
         .andReturn(new HashMap<String, Service>() {
           {
@@ -2157,7 +2186,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     setupKerberosDescriptor(kerberosDescriptor, 2);
 
     RecommendationResponse.BlueprintConfigurations coreSiteRecommendation = createNiceMock(RecommendationResponse
-      .BlueprintConfigurations.class);
+        .BlueprintConfigurations.class);
     expect(coreSiteRecommendation.getProperties()).andReturn(Collections.singletonMap("newPropertyRecommendation", "newPropertyRecommendation"));
 
     RecommendationResponse.BlueprintConfigurations newTypeRecommendation = createNiceMock(RecommendationResponse.BlueprintConfigurations.class);
@@ -2248,7 +2277,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     serviceComponentHostMap.put("COMPONENT2B", new HashSet<String>(Arrays.asList("hostB", "hostC")));
     serviceComponentHostMap.put("COMPONEN3A", Collections.singleton("hostA"));
 
-    final Cluster cluster = createMockCluster(hosts, SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
+    final Cluster cluster = createMockCluster("c1", hosts, SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
     expect(cluster.getServices()).andReturn(services).anyTimes();
     expect(cluster.getServiceComponentHostMap(anyObject(Set.class), anyObject(Set.class))).andReturn(serviceComponentHostMap).anyTimes();
 
@@ -2286,7 +2315,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     // Needed by infrastructure
     injector.getInstance(AmbariMetaInfo.class).init();
 
-    HashMap<String,Set<String>> installedServices1 = new HashMap<String, Set<String>>();
+    HashMap<String, Set<String>> installedServices1 = new HashMap<String, Set<String>>();
     installedServices1.put("SERVICE1", new HashSet<String>(Arrays.asList("COMPONENT1A", "COMPONENT1B")));
     installedServices1.put("SERVICE2", new HashSet<String>(Arrays.asList("COMPONENT2A", "COMPONENT2B")));
     installedServices1.put("SERVICE3", Collections.singleton("COMPONENT3A"));
@@ -2294,7 +2323,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     Map<String, Map<String, String>> updates1 = kerberosHelper.getServiceConfigurationUpdates(
         cluster, existingConfigurations, installedServices1, null, null, true, true);
 
-    HashMap<String,Set<String>> installedServices2 = new HashMap<String, Set<String>>();
+    HashMap<String, Set<String>> installedServices2 = new HashMap<String, Set<String>>();
     installedServices2.put("SERVICE1", new HashSet<String>(Arrays.asList("COMPONENT1A", "COMPONENT1B")));
     installedServices2.put("SERVICE3", Collections.singleton("COMPONENT3A"));
 
@@ -2380,7 +2409,7 @@ public class KerberosHelperTest extends EasyMockSupport {
             put("realm", "FOOBAR.COM");
             put("case_insensitive_username_rules", "false");
             put("create_ambari_principal", "false");
-        }
+          }
         });
         put("", new HashMap<String, String>() {
           {
@@ -2413,18 +2442,51 @@ public class KerberosHelperTest extends EasyMockSupport {
 
   @Test
   public void testEnsureHeadlessIdentities() throws Exception {
+    testEnsureHeadlessIdentities(false, false);
+  }
+
+  @Test
+  public void testEnsureHeadlessAndAmbariIdentitiesAsUser() throws Exception {
+    testEnsureHeadlessIdentities(true, false);
+  }
+
+  @Test
+  public void testEnsureHeadlessAndAmbariIdentitiesAsService() throws Exception {
+    testEnsureHeadlessIdentities(true, true);
+  }
+
+  private void testEnsureHeadlessIdentities(boolean createAmbariPrincipal, boolean ambariServerPrincipalAsService) throws Exception {
+    String clusterName = "c1";
+    String realm = "EXAMPLE.COM";
+    String ambariServerHostname = StageUtils.getHostName();
+    String ambariServerPrincipalName;
+    String ambariServerKeytabFilePath = new File("ambari.server.keytab").getAbsolutePath();
+    KerberosPrincipalType ambariServerPrincipalType;
+    String ambariServerPrincipalNameExpected;
+
+    if(ambariServerPrincipalAsService) {
+      ambariServerPrincipalName = "ambari-server-${cluster_name}/_HOST@${realm}";
+      ambariServerPrincipalType = KerberosPrincipalType.SERVICE;
+      ambariServerPrincipalNameExpected = String.format("ambari-server-%s/%s@%s", clusterName, ambariServerHostname, realm);
+    }
+    else {
+      ambariServerPrincipalName = "ambari-server-${cluster_name}@${realm}";
+      ambariServerPrincipalType = KerberosPrincipalType.USER;
+      ambariServerPrincipalNameExpected = String.format("ambari-server-%s@%s", clusterName, realm);
+    }
+
     Map<String, String> propertiesKrb5Conf = new HashMap<String, String>();
 
     Map<String, String> propertiesKerberosEnv = new HashMap<String, String>();
-    propertiesKerberosEnv.put("realm", "EXAMPLE.COM");
+    propertiesKerberosEnv.put("realm", realm);
     propertiesKerberosEnv.put("kdc_type", "mit-kdc");
     propertiesKerberosEnv.put("password_length", "20");
     propertiesKerberosEnv.put("password_min_lowercase_letters", "1");
     propertiesKerberosEnv.put("password_min_uppercase_letters", "1");
     propertiesKerberosEnv.put("password_min_digits", "1");
     propertiesKerberosEnv.put("password_min_punctuation", "0");
-    propertiesKerberosEnv.put("password_min_whitespace","0");
-    propertiesKerberosEnv.put("create_ambari_principal", "false");
+    propertiesKerberosEnv.put("password_min_whitespace", "0");
+    propertiesKerberosEnv.put("create_ambari_principal", (createAmbariPrincipal) ? "true" : "false");
 
     Config configKrb5Conf = createMock(Config.class);
     expect(configKrb5Conf.getProperties()).andReturn(propertiesKrb5Conf).times(1);
@@ -2455,7 +2517,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     servicesMap.put("SERVICE1", service1);
     servicesMap.put("SERVICE2", service2);
 
-    Cluster cluster = createMockCluster(Arrays.asList(host1, host2, host3), SecurityType.KERBEROS, configKrb5Conf, configKerberosEnv);
+    Cluster cluster = createMockCluster(clusterName, Arrays.asList(host1, host2, host3), SecurityType.KERBEROS, configKrb5Conf, configKerberosEnv);
     expect(cluster.getServices()).andReturn(servicesMap).times(1);
 
     Map<String, String> kerberosDescriptorProperties = new HashMap<String, String>();
@@ -2514,6 +2576,36 @@ public class KerberosHelperTest extends EasyMockSupport {
     expect(kerberosDescriptor.getService("SERVICE1")).andReturn(service1KerberosDescriptor).times(1);
     expect(kerberosDescriptor.getService("SERVICE2")).andReturn(service2KerberosDescriptor).times(1);
 
+    if (createAmbariPrincipal) {
+      String spnegoPrincipalNameExpected = String.format("HTTP/%s@%s", ambariServerHostname, realm);
+
+      KerberosIdentityDescriptor ambariIdentity = createMockIdentityDescriptor(
+          KerberosHelper.AMBARI_IDENTITY_NAME,
+          createMockPrincipalDescriptor(ambariServerPrincipalName, ambariServerPrincipalType, "ambari", null),
+          createMockKeytabDescriptor(ambariServerKeytabFilePath, null));
+
+      KerberosIdentityDescriptor spnegoIdentity = createMockIdentityDescriptor(
+          KerberosHelper.SPNEGO_IDENTITY_NAME,
+          createMockPrincipalDescriptor("HTTP/_HOST@${realm}", KerberosPrincipalType.SERVICE, null, null),
+          createMockKeytabDescriptor("spnego.service.keytab", null));
+
+      expect(kerberosDescriptor.getIdentity(KerberosHelper.AMBARI_IDENTITY_NAME)).andReturn(ambariIdentity).once();
+      expect(kerberosDescriptor.getIdentity(KerberosHelper.SPNEGO_IDENTITY_NAME)).andReturn(spnegoIdentity).once();
+
+      ConfigureAmbariIdentitiesServerAction configureAmbariIdentitiesServerAction = injector.getInstance(ConfigureAmbariIdentitiesServerAction.class);
+      expect(configureAmbariIdentitiesServerAction.installAmbariServerIdentity(eq(ambariServerPrincipalNameExpected), anyString(), eq(ambariServerKeytabFilePath),
+          eq("user1"), eq(true), eq(true), eq("groupA"), eq(true), eq(false), (ActionLog) eq(null)))
+          .andReturn(true)
+          .once();
+      expect(configureAmbariIdentitiesServerAction.installAmbariServerIdentity(eq(spnegoPrincipalNameExpected), anyString(), eq("spnego.service.keytab"),
+          eq("user1"), eq(true), eq(true), eq("groupA"), eq(true), eq(false), (ActionLog) eq(null)))
+          .andReturn(true)
+          .once();
+
+      configureAmbariIdentitiesServerAction.configureJAAS(ambariServerPrincipalNameExpected, ambariServerKeytabFilePath, null);
+      expectLastCall().once();
+    }
+
     setupKerberosDescriptor(kerberosDescriptor, 1);
 
     Map<String, Map<String, String>> existingConfigurations = new HashMap<String, Map<String, String>>();
@@ -2530,15 +2622,30 @@ public class KerberosHelperTest extends EasyMockSupport {
     Capture<? extends String> capturePrincipalForKeytab = newCapture(CaptureType.ALL);
 
     CreatePrincipalsServerAction createPrincipalsServerAction = injector.getInstance(CreatePrincipalsServerAction.class);
-    expect(createPrincipalsServerAction.createPrincipal(capture(capturePrincipal), eq(false), anyObject(Map.class),  anyObject(KerberosOperationHandler.class), eq(false), isNull(ActionLog.class)))
+    expect(createPrincipalsServerAction.createPrincipal(capture(capturePrincipal), eq(false), anyObject(Map.class), anyObject(KerberosOperationHandler.class), eq(false), isNull(ActionLog.class)))
         .andReturn(new CreatePrincipalsServerAction.CreatePrincipalResult("anything", "password", 1))
         .times(3);
 
+    if(createAmbariPrincipal) {
+      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))
+            .times(2);
+      } else {
+        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))
+            .times(1);
+        expect(createPrincipalsServerAction.createPrincipal(capture(capturePrincipal), eq(false), anyObject(Map.class), anyObject(KerberosOperationHandler.class), eq(false), isNull(ActionLog.class)))
+            .andReturn(new CreatePrincipalsServerAction.CreatePrincipalResult("anything", "password", 1))
+            .times(1);
+      }
+    }
+
     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(3);
-    
+        .times(createAmbariPrincipal ? 5 : 3);
+
     replayAll();
 
     AmbariMetaInfo ambariMetaInfo = injector.getInstance(AmbariMetaInfo.class);
@@ -2546,7 +2653,7 @@ public class KerberosHelperTest extends EasyMockSupport {
 
     CredentialStoreService credentialStoreService = injector.getInstance(CredentialStoreService.class);
     credentialStoreService.setCredential(cluster.getClusterName(), KerberosHelper.KDC_ADMINISTRATOR_CREDENTIAL_ALIAS,
-      new PrincipalKeyCredential("principal", "password"), CredentialStoreType.TEMPORARY);
+        new PrincipalKeyCredential("principal", "password"), CredentialStoreType.TEMPORARY);
 
     KerberosHelper kerberosHelper = injector.getInstance(KerberosHelper.class);
     kerberosHelper.ensureHeadlessIdentities(cluster, existingConfigurations, services);
@@ -2554,16 +2661,26 @@ public class KerberosHelperTest extends EasyMockSupport {
     verifyAll();
 
     List<? extends String> capturedPrincipals = capturePrincipal.getValues();
-    assertEquals(3, capturedPrincipals.size());
+    assertEquals(createAmbariPrincipal ? 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(3, capturedPrincipalsForKeytab.size());
+    assertEquals(createAmbariPrincipal ? 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) {
+      String spnegoPrincipalName = String.format("HTTP/%s@EXAMPLE.COM", ambariServerHostname);
+
+      assertTrue(capturedPrincipals.contains(ambariServerPrincipalNameExpected));
+      assertTrue(capturedPrincipals.contains(spnegoPrincipalName));
+
+      assertTrue(capturedPrincipalsForKeytab.contains(ambariServerPrincipalNameExpected));
+      assertTrue(capturedPrincipalsForKeytab.contains(spnegoPrincipalName));
+    }
   }
 
   /**
@@ -2598,7 +2715,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     Map<String, Service> servicesMap = new HashMap<String, Service>();
     servicesMap.put("SERVICE1", service1);
 
-    Cluster cluster = createMockCluster(Arrays.asList(host1), SecurityType.KERBEROS, configKrb5Conf, configKerberosEnv);
+    Cluster cluster = createMockCluster("c1", Arrays.asList(host1), SecurityType.KERBEROS, configKrb5Conf, configKerberosEnv);
     expect(cluster.getServices()).andReturn(servicesMap).times(1);
 
     Map<String, String> kerberosDescriptorProperties = new HashMap<String, String>();
@@ -2769,7 +2886,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     final Config krb5ConfConfig = createMock(Config.class);
     expect(krb5ConfConfig.getProperties()).andReturn(krb5ConfProperties).anyTimes();
 
-    final Cluster cluster = createMockCluster(Arrays.asList(hostA, hostB, hostC), SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
+    final Cluster cluster = createMockCluster("c1", Arrays.asList(hostA, hostB, hostC), SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
     expect(cluster.getServices())
         .andReturn(new HashMap<String, Service>() {
           {
@@ -3014,7 +3131,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     final Config krb5ConfConfig = createMock(Config.class);
     expect(krb5ConfConfig.getProperties()).andReturn(krb5ConfProperties).anyTimes();
 
-    final Cluster cluster = createMockCluster(Collections.singleton(host), SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
+    final Cluster cluster = createMockCluster("c1", Collections.singleton(host), SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
     expect(cluster.getServices())
         .andReturn(new HashMap<String, Service>() {
           {
@@ -3146,10 +3263,10 @@ public class KerberosHelperTest extends EasyMockSupport {
     KerberosHelper kerberosHelper = injector.getInstance(KerberosHelper.class);
     boolean managingIdentities = !Boolean.FALSE.equals(manageIdentities);
 
-    final Map<String, String> kerberosEnvProperties = new HashMap<String,String>();
-    kerberosEnvProperties.put("kdc_type","mit-kdc");
-    kerberosEnvProperties.put("realm","FOOBAR.COM");
-    kerberosEnvProperties.put("manage_identities","FOOBAR.COM");
+    final Map<String, String> kerberosEnvProperties = new HashMap<String, String>();
+    kerberosEnvProperties.put("kdc_type", "mit-kdc");
+    kerberosEnvProperties.put("realm", "FOOBAR.COM");
+    kerberosEnvProperties.put("manage_identities", "FOOBAR.COM");
     kerberosEnvProperties.put("manage_identities",
         (manageIdentities == null)
             ? null
@@ -3378,7 +3495,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     final Config krb5ConfConfig = createMock(Config.class);
     expect(krb5ConfConfig.getProperties()).andReturn(krb5ConfProperties).anyTimes();
 
-    final Cluster cluster = createMockCluster(Collections.singleton(host), SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
+    final Cluster cluster = createMockCluster("c1", Collections.singleton(host), SecurityType.KERBEROS, krb5ConfConfig, kerberosEnvConfig);
     expect(cluster.getServices())
         .andReturn(new HashMap<String, Service>() {
           {
@@ -3558,8 +3675,8 @@ public class KerberosHelperTest extends EasyMockSupport {
         })
         .anyTimes();
     expect(cluster.getServiceComponentHosts(InetAddress.getLocalHost().getCanonicalHostName().toLowerCase()))
-      .andReturn(new ArrayList<ServiceComponentHost>())
-      .anyTimes();
+        .andReturn(new ArrayList<ServiceComponentHost>())
+        .anyTimes();
 
     final Map<String, String> kerberosEnvProperties = new HashMap<String, String>() {
       {
@@ -3579,12 +3696,12 @@ public class KerberosHelperTest extends EasyMockSupport {
     expect(krb5ConfConfig.getProperties()).andReturn(krb5ConfProperties).anyTimes();
 
     expect(cluster.getDesiredConfigByType("krb5-conf"))
-      .andReturn(krb5ConfConfig)
-      .anyTimes();
+        .andReturn(krb5ConfConfig)
+        .anyTimes();
 
     expect(cluster.getDesiredConfigByType("kerberos-env"))
-      .andReturn(kerberosEnvConfig)
-      .anyTimes();
+        .andReturn(kerberosEnvConfig)
+        .anyTimes();
 
     expect(cluster.getCurrentStackVersion())
         .andReturn(new StackId("HDP", "2.2"))
@@ -3786,6 +3903,10 @@ public class KerberosHelperTest extends EasyMockSupport {
     KerberosKeytabDescriptor descriptor = createMock(KerberosKeytabDescriptor.class);
     expect(descriptor.getFile()).andReturn(file).anyTimes();
     expect(descriptor.getConfiguration()).andReturn(configuration).anyTimes();
+    expect(descriptor.getOwnerName()).andReturn("user1").anyTimes();
+    expect(descriptor.getOwnerAccess()).andReturn("rw").anyTimes();
+    expect(descriptor.getGroupName()).andReturn("groupA").anyTimes();
+    expect(descriptor.getGroupAccess()).andReturn("r").anyTimes();
     return descriptor;
   }
 
@@ -3866,14 +3987,14 @@ public class KerberosHelperTest extends EasyMockSupport {
     return host;
   }
 
-  private Cluster createMockCluster(Collection<Host> hosts, SecurityType securityType, Config krb5ConfConfig, Config kerberosEnvConfig) {
+  private Cluster createMockCluster(String clusterName, Collection<Host> hosts, SecurityType securityType, Config krb5ConfConfig, Config kerberosEnvConfig) {
     Cluster cluster = createMock(Cluster.class);
     expect(cluster.getHosts()).andReturn(hosts).anyTimes();
     expect(cluster.getClusterId()).andReturn(1L).anyTimes();
     expect(cluster.getSecurityType()).andReturn(securityType).anyTimes();
     expect(cluster.getDesiredConfigByType("krb5-conf")).andReturn(krb5ConfConfig).anyTimes();
     expect(cluster.getDesiredConfigByType("kerberos-env")).andReturn(kerberosEnvConfig).anyTimes();
-    expect(cluster.getClusterName()).andReturn("c1").anyTimes();
+    expect(cluster.getClusterName()).andReturn(clusterName).anyTimes();
     expect(cluster.getCurrentStackVersion())
         .andReturn(new StackId("HDP", "2.2"))
         .anyTimes();