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 2018/07/30 14:04:47 UTC

[ambari] branch branch-2.7 updated: [AMBARI-24319] Multiple alerts after HDFS service only regenerate keytabs, as keytabs are out of sync

This is an automated email from the ASF dual-hosted git repository.

rlevas pushed a commit to branch branch-2.7
in repository https://gitbox.apache.org/repos/asf/ambari.git


The following commit(s) were added to refs/heads/branch-2.7 by this push:
     new ec9ac66  [AMBARI-24319] Multiple alerts after HDFS service only regenerate keytabs, as keytabs are out of sync
ec9ac66 is described below

commit ec9ac6670be53c77920ae9c40948ad6596f6f78b
Author: Robert Levas <rl...@hortonworks.com>
AuthorDate: Sun Jul 29 07:39:27 2018 -0400

    [AMBARI-24319] Multiple alerts after HDFS service only regenerate keytabs, as keytabs are out of sync
---
 .../server/controller/DeleteIdentityHandler.java   |  2 +-
 .../server/controller/KerberosHelperImpl.java      |  7 ++--
 .../AbstractPrepareKerberosServerAction.java       |  2 +-
 .../serveraction/kerberos/CleanupServerAction.java |  3 ++
 .../ConfigureAmbariIdentitiesServerAction.java     |  7 +++-
 .../kerberos/CreateKeytabFilesServerAction.java    | 10 ++++-
 .../kerberos/CreatePrincipalsServerAction.java     | 11 ++++-
 .../kerberos/DestroyPrincipalsServerAction.java    | 10 ++++-
 .../kerberos/FinalizeKerberosServerAction.java     |  3 ++
 .../kerberos/KerberosServerAction.java             | 49 +++++++++++++++++-----
 .../PrepareEnableKerberosServerAction.java         |  2 +-
 .../kerberos/KerberosServerActionTest.java         |  1 +
 ambari-web/app/controllers/main/host/details.js    |  2 +-
 ambari-web/app/utils/ajax/ajax.js                  |  4 +-
 14 files changed, 87 insertions(+), 26 deletions(-)

diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/DeleteIdentityHandler.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/DeleteIdentityHandler.java
index 36cc169..927e2d5 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/DeleteIdentityHandler.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/DeleteIdentityHandler.java
@@ -321,7 +321,7 @@ class DeleteIdentityHandler {
     }
 
     @Override
-    protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal, KerberosOperationHandler operationHandler, Map<String, String> kerberosConfiguration, Map<String, Object> requestSharedDataContext) throws AmbariException {
+    protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal, KerberosOperationHandler operationHandler, Map<String, String> kerberosConfiguration, boolean includedInFilter, Map<String, Object> requestSharedDataContext) throws AmbariException {
       return null;
     }
   }
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 b08ddbe..0d359de 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
@@ -138,7 +138,6 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.gson.JsonArray;
 import com.google.gson.JsonObject;
@@ -1867,7 +1866,8 @@ public class KerberosHelperImpl implements KerberosHelper {
 
                   String uniqueKey = String.format("%s|%s", principal, (keytabFile == null) ? "" : keytabFile);
 
-                  if (!hostActiveIdentities.containsKey(uniqueKey)) {
+                  if (!hostActiveIdentities.containsKey(uniqueKey) ||
+                      (StringUtils.isNotBlank(hostActiveIdentities.get(uniqueKey).getReference()) && StringUtils.isBlank(identity.getReference()))) {
                     KerberosPrincipalType principalType = principalDescriptor.getType();
 
                     // Assume the principal is a service principal if not specified
@@ -2468,8 +2468,7 @@ public class KerberosHelperImpl implements KerberosHelper {
       handler.createStages(cluster,
         clusterHostInfoJson, hostParamsJson, event, roleCommandOrder, kerberosDetails,
         dataDirectory, requestStageContainer, serviceComponentHostsToProcess,
-        Collections.singletonMap("KERBEROS", Lists.newArrayList("KERBEROS_CLIENT")),
-        null, Sets.newHashSet(principal), hostsWithValidKerberosClient);
+        null, null, Sets.newHashSet(principal), hostsWithValidKerberosClient);
 
 
       handler.addFinalizeOperationStage(cluster, clusterHostInfoJson, hostParamsJson, event,
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java
index ce1d808..72a4e14 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java
@@ -72,7 +72,7 @@ public abstract class AbstractPrepareKerberosServerAction extends KerberosServer
   private ConfigHelper configHelper;
 
   @Override
-  protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal, KerberosOperationHandler operationHandler, Map<String, String> kerberosConfiguration, Map<String, Object> requestSharedDataContext) throws AmbariException {
+  protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal, KerberosOperationHandler operationHandler, Map<String, String> kerberosConfiguration, boolean includedInFilter, Map<String, Object> requestSharedDataContext) throws AmbariException {
     throw new UnsupportedOperationException();
   }
 
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CleanupServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CleanupServerAction.java
index 1b7d128..c2f82d9 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CleanupServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CleanupServerAction.java
@@ -64,6 +64,8 @@ public class CleanupServerAction extends KerberosServerAction {
    *                                 tasks for specific Kerberos implementations
    *                                 (MIT, Active Directory, etc...)
    * @param kerberosConfiguration    a Map of configuration properties from kerberos-env
+   * @param includedInFilter         a Boolean value indicating whather the principal is included in
+   *                                 the current filter or not
    * @param requestSharedDataContext a Map to be used a shared data among all ServerActions related
    *                                 to a given request
    * @return null, always
@@ -73,6 +75,7 @@ public class CleanupServerAction extends KerberosServerAction {
   protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal,
                                           KerberosOperationHandler operationHandler,
                                           Map<String, String> kerberosConfiguration,
+                                          boolean includedInFilter,
                                           Map<String, Object> requestSharedDataContext)
       throws AmbariException {
     return null;
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 5c0f319..87aa3e7 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
@@ -54,7 +54,7 @@ import com.google.inject.Inject;
  * This class mainly relies on the KerberosServerAction to iterate through metadata identifying
  * the Kerberos keytab files that need to be created. For each identity in the metadata, this
  * implementation's
- * {@link KerberosServerAction#processIdentity(ResolvedKerberosPrincipal, KerberosOperationHandler, Map, Map)}
+ * {@link KerberosServerAction#processIdentity(ResolvedKerberosPrincipal, KerberosOperationHandler, Map, boolean, Map)}
  * is invoked attempting the creation of the relevant keytab file.
  */
 public class ConfigureAmbariIdentitiesServerAction extends KerberosServerAction {
@@ -108,6 +108,8 @@ public class ConfigureAmbariIdentitiesServerAction extends KerberosServerAction
    *                                 tasks for specific Kerberos implementations
    *                                 (MIT, Active Directory, etc...)
    * @param kerberosConfiguration    a Map of configuration properties from kerberos-env
+   * @param includedInFilter         a Boolean value indicating whather the principal is included in
+   *                                 the current filter or not
    * @param requestSharedDataContext a Map to be used a shared data among all ServerActions related
    *                                 to a given request  @return a CommandReport, indicating an error
    *                                 condition; or null, indicating a success condition
@@ -117,11 +119,12 @@ public class ConfigureAmbariIdentitiesServerAction extends KerberosServerAction
   protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal,
                                           KerberosOperationHandler operationHandler,
                                           Map<String, String> kerberosConfiguration,
+                                          boolean includedInFilter,
                                           Map<String, Object> requestSharedDataContext)
       throws AmbariException {
     CommandReport commandReport = null;
 
-    if (resolvedPrincipal != null && StageUtils.getHostName().equals(resolvedPrincipal.getHostName())) {
+    if (includedInFilter && resolvedPrincipal != null && StageUtils.getHostName().equals(resolvedPrincipal.getHostName())) {
       final String hostName = resolvedPrincipal.getHostName();
       final String dataDirectory = getDataDirectoryPath();
       for (Map.Entry<String, String> serviceMappingEntry : resolvedPrincipal.getServiceMapping().entries()) {
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CreateKeytabFilesServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CreateKeytabFilesServerAction.java
index 0e3ad5f..351e861 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CreateKeytabFilesServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CreateKeytabFilesServerAction.java
@@ -53,7 +53,7 @@ import com.google.inject.Inject;
  * This class mainly relies on the KerberosServerAction to iterate through metadata identifying
  * the Kerberos keytab files that need to be created. For each identity in the metadata, this
  * implementation's
- * {@link KerberosServerAction#processIdentity(ResolvedKerberosPrincipal, KerberosOperationHandler, Map, Map)}
+ * {@link KerberosServerAction#processIdentity(ResolvedKerberosPrincipal, KerberosOperationHandler, Map, boolean, Map)}
  * is invoked attempting the creation of the relevant keytab file.
  */
 public class CreateKeytabFilesServerAction extends KerberosServerAction {
@@ -133,6 +133,8 @@ public class CreateKeytabFilesServerAction extends KerberosServerAction {
    *                                 tasks for specific Kerberos implementations
    *                                 (MIT, Active Directory, etc...)
    * @param kerberosConfiguration    a Map of configuration properties from kerberos-env
+   * @param includedInFilter         a Boolean value indicating whather the principal is included in
+   *                                 the current filter or not
    * @param requestSharedDataContext a Map to be used a shared data among all ServerActions related
    *                                 to a given request  @return a CommandReport, indicating an error
    *                                 condition; or null, indicating a success condition
@@ -142,6 +144,7 @@ public class CreateKeytabFilesServerAction extends KerberosServerAction {
   protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal,
                                           KerberosOperationHandler operationHandler,
                                           Map<String, String> kerberosConfiguration,
+                                          boolean includedInFilter,
                                           Map<String, Object> requestSharedDataContext)
       throws AmbariException {
 
@@ -206,6 +209,11 @@ public class CreateKeytabFilesServerAction extends KerberosServerAction {
 
                 boolean regenerateKeytabs = getOperationType(getCommandParameters()) == OperationType.RECREATE_ALL;
 
+                if(!includedInFilter) {
+                  // If this principal is to be filtered out, skip... unless is has not yet been created...
+                  regenerateKeytabs = false;
+                }
+
                 KerberosPrincipalEntity principalEntity = kerberosPrincipalDAO.find(resolvedPrincipal.getPrincipal());
                 String cachedKeytabPath = (principalEntity == null) ? null : principalEntity.getCachedKeytabPath();
 
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CreatePrincipalsServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CreatePrincipalsServerAction.java
index 6e01a42..175b0ac 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CreatePrincipalsServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/CreatePrincipalsServerAction.java
@@ -46,7 +46,7 @@ import com.google.inject.Inject;
  * <p/>
  * This class mainly relies on the KerberosServerAction to iterate through metadata identifying
  * the Kerberos principals that need to be created. For each identity in the metadata, this implementation's
- * {@link KerberosServerAction#processIdentity(ResolvedKerberosPrincipal, KerberosOperationHandler, Map, Map)}
+ * {@link KerberosServerAction#processIdentity(ResolvedKerberosPrincipal, KerberosOperationHandler, Map, boolean, Map)}
  * is invoked attempting the creation of the relevant principal.
  */
 public class CreatePrincipalsServerAction extends KerberosServerAction {
@@ -109,6 +109,8 @@ public class CreatePrincipalsServerAction extends KerberosServerAction {
    *                                 tasks for specific Kerberos implementations
    *                                 (MIT, Active Directory, etc...)
    * @param kerberosConfiguration    a Map of configuration properties from kerberos-env
+   * @param includedInFilter         a Boolean value indicating whather the principal is included in
+   *                                 the current filter or not
    * @param requestSharedDataContext a Map to be used as shared data among all ServerActions related
    *                                 to a given request  @return a CommandReport, indicating an error
    *                                 condition; or null, indicating a success condition
@@ -118,6 +120,7 @@ public class CreatePrincipalsServerAction extends KerberosServerAction {
   protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal,
                                           KerberosOperationHandler operationHandler,
                                           Map<String, String> kerberosConfiguration,
+                                          boolean includedInFilter,
                                           Map<String, Object> requestSharedDataContext)
       throws AmbariException {
     CommandReport commandReport = null;
@@ -133,6 +136,12 @@ public class CreatePrincipalsServerAction extends KerberosServerAction {
 
       boolean regenerateKeytabs = getOperationType(getCommandParameters()) == OperationType.RECREATE_ALL;
       boolean servicePrincipal = resolvedPrincipal.isService();
+
+      if(!includedInFilter) {
+        // If this principal is to be filtered out, skip... unless is has not yet been created...
+        regenerateKeytabs = false;
+      }
+
       if (regenerateKeytabs) {
         // force recreation of principal due to keytab regeneration
         // regenerate only service principals if request filtered by hosts
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/DestroyPrincipalsServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/DestroyPrincipalsServerAction.java
index e02ca5a..a57f528 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/DestroyPrincipalsServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/DestroyPrincipalsServerAction.java
@@ -48,7 +48,7 @@ import com.google.inject.Inject;
  * This class mainly relies on the KerberosServerAction to iterate through metadata identifying
  * the Kerberos principals that need to be removed from the relevant KDC. For each identity in the
  * metadata, this implementation's
- * {@link KerberosServerAction#processIdentity(ResolvedKerberosPrincipal, KerberosOperationHandler, Map, Map)}
+ * {@link KerberosServerAction#processIdentity(ResolvedKerberosPrincipal, KerberosOperationHandler, Map, boolean, Map)}
  * is invoked attempting the removal of the relevant principal.
  */
 public class DestroyPrincipalsServerAction extends KerberosServerAction {
@@ -97,6 +97,8 @@ public class DestroyPrincipalsServerAction extends KerberosServerAction {
    *                                 tasks for specific Kerberos implementations
    *                                 (MIT, Active Directory, etc...)
    * @param kerberosConfiguration    a Map of configuration properties from kerberos-env
+   * @param includedInFilter         a Boolean value indicating whather the principal is included in
+   *                                 the current filter or not
    * @param requestSharedDataContext a Map to be used a shared data among all ServerActions related
    *                                 to a given request  @return a CommandReport, indicating an error
    *                                 condition; or null, indicating a success condition
@@ -106,9 +108,15 @@ public class DestroyPrincipalsServerAction extends KerberosServerAction {
   protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal,
                                           KerberosOperationHandler operationHandler,
                                           Map<String, String> kerberosConfiguration,
+                                          boolean includedInFilter,
                                           Map<String, Object> requestSharedDataContext)
       throws AmbariException {
 
+    if(!includedInFilter) {
+      // If this principal is to be filtered out, skip it
+      return null;
+    }
+
     // Only process this principal if we haven't already processed it
     if (!seenPrincipals.contains(resolvedPrincipal.getPrincipal())) {
       seenPrincipals.add(resolvedPrincipal.getPrincipal());
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 6823d16..b3b3082 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
@@ -60,6 +60,8 @@ public class FinalizeKerberosServerAction extends KerberosServerAction {
    *                                 tasks for specific Kerberos implementations
    *                                 (MIT, Active Directory, etc...)
    * @param kerberosConfiguration    a Map of configuration properties from kerberos-env
+   * @param includedInFilter         a Boolean value indicating whather the principal is included in
+   *                                 the current filter or not
    * @param requestSharedDataContext a Map to be used a shared data among all ServerActions related
    *                                 to a given request  @return null, always
    * @throws AmbariException
@@ -68,6 +70,7 @@ public class FinalizeKerberosServerAction extends KerberosServerAction {
   protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal,
                                           KerberosOperationHandler operationHandler,
                                           Map<String, String> kerberosConfiguration,
+                                          boolean includedInFilter,
                                           Map<String, Object> requestSharedDataContext)
       throws AmbariException {
 
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/KerberosServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/KerberosServerAction.java
index 904fd01..394c7c5 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/KerberosServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/KerberosServerAction.java
@@ -21,6 +21,7 @@ package org.apache.ambari.server.serveraction.kerberos;
 import java.io.File;
 import java.io.IOException;
 import java.lang.reflect.Type;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
@@ -41,6 +42,7 @@ import org.apache.ambari.server.serveraction.kerberos.stageutils.ResolvedKerbero
 import org.apache.ambari.server.serveraction.kerberos.stageutils.ResolvedKerberosPrincipal;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
+import org.apache.ambari.server.state.kerberos.KerberosIdentityDescriptor;
 import org.apache.ambari.server.utils.StageUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
@@ -178,12 +180,6 @@ public abstract class KerberosServerAction extends AbstractServerAction {
   private KerberosOperationHandlerFactory kerberosOperationHandlerFactory;
 
   /**
-   * The KerberosIdentityDataFileReaderFactory to use to obtain KerberosIdentityDataFileReader instances
-   */
-  @Inject
-  private KerberosIdentityDataFileReaderFactory kerberosIdentityDataFileReaderFactory;
-
-  /**
    * KerberosHelper
    */
   @Inject
@@ -420,7 +416,7 @@ public abstract class KerberosServerAction extends AbstractServerAction {
    * <p/>
    * Using {@link #getHostFilter()}, {@link #getIdentityFilter()} and {@link #getServiceComponentFilter()} it retrieve
    * list of filtered keytabs and their principals and process each principal using
-   * {@link #processIdentity(ResolvedKerberosPrincipal, KerberosOperationHandler, Map, Map)}.
+   * {@link #processIdentity(ResolvedKerberosPrincipal, KerberosOperationHandler, Map, boolean, Map)}.
    *
    * @param requestSharedDataContext a Map to be used a shared data among all ServerActions related
    *                                 to a given request
@@ -455,11 +451,13 @@ public abstract class KerberosServerAction extends AbstractServerAction {
       }
 
       try {
-        for (ResolvedKerberosKeytab rkk : kerberosKeytabController.getFilteredKeytabs((Map<String, Collection<String>>) getServiceComponentFilter(), getHostFilter(), getIdentityFilter())) {
+        final Map<String, Collection<String>> serviceComponentFilter = (Map<String, Collection<String>>) getServiceComponentFilter();
+        final Collection<KerberosIdentityDescriptor> serviceIdentities = serviceComponentFilter == null ? null : calculateServiceIdentities(getClusterName(), serviceComponentFilter);
+        for (ResolvedKerberosKeytab rkk : kerberosKeytabController.getFilteredKeytabs(serviceComponentFilter, getHostFilter(), getIdentityFilter())) {
           for (ResolvedKerberosPrincipal principal : rkk.getPrincipals()) {
-            commandReport = processIdentity(principal, handler, kerberosConfiguration, requestSharedDataContext);
-            // If the principal processor returns a CommandReport, than it is time to stop since
-            // an error condition has probably occurred, else all is assumed to be well.
+            commandReport = processIdentity(principal, handler, kerberosConfiguration, isRelevantIdentity(serviceIdentities, principal), requestSharedDataContext);
+            // If the principal processor returns a CommandReport, than it is time to stop
+            // since an error condition has probably occurred, else all is assumed to be well.
             if (commandReport != null) {
               break;
             }
@@ -487,6 +485,32 @@ public abstract class KerberosServerAction extends AbstractServerAction {
         : commandReport;
   }
 
+  private boolean isRelevantIdentity(Collection<KerberosIdentityDescriptor> serviceIdentities, ResolvedKerberosPrincipal principal) {
+    if (serviceIdentities != null) {
+      boolean hasValidIdentity = false;
+      for (KerberosIdentityDescriptor serviceIdentity : serviceIdentities) {
+        if (principal.getPrincipal().equals(serviceIdentity.getPrincipalDescriptor().getName()) && StringUtils.isBlank(serviceIdentity.getReference())) {
+          hasValidIdentity = true;
+          break;
+        }
+      }
+      return hasValidIdentity;
+    }
+
+    return true;
+  }
+
+  private Collection<KerberosIdentityDescriptor> calculateServiceIdentities(String clusterName, Map<String, Collection<String>> serviceComponentFilter)
+      throws AmbariException {
+    final Collection<KerberosIdentityDescriptor> serviceIdentities = new ArrayList<>();
+    for (String service : serviceComponentFilter.keySet()) {
+      for (Collection<KerberosIdentityDescriptor> activeIdentities : kerberosHelper.getActiveIdentities(clusterName, null, service, null, true).values()) {
+        serviceIdentities.addAll(activeIdentities);
+      }
+    }
+    return serviceIdentities;
+  }
+
   /**
    * Processes an identity as necessary.
    * <p/>
@@ -499,6 +523,8 @@ public abstract class KerberosServerAction extends AbstractServerAction {
    *                                 tasks for specific Kerberos implementations
    *                                 (MIT, Active Directory, etc...)
    * @param kerberosConfiguration    a Map of configuration properties from kerberos-env
+   * @param includedInFilter         a Boolean value indicating whather the principal is included in
+   *                                 the current filter or not
    * @param requestSharedDataContext a Map to be used a shared data among all ServerActions related
    *                                 to a given request  @return a CommandReport, indicating an error
    *                                 condition; or null, indicating a success condition
@@ -507,6 +533,7 @@ public abstract class KerberosServerAction extends AbstractServerAction {
   protected abstract CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal,
                                                    KerberosOperationHandler operationHandler,
                                                    Map<String, String> kerberosConfiguration,
+                                                   boolean includedInFilter,
                                                    Map<String, Object> requestSharedDataContext)
       throws AmbariException;
 
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareEnableKerberosServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareEnableKerberosServerAction.java
index f9b9717..a130df4 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareEnableKerberosServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareEnableKerberosServerAction.java
@@ -134,7 +134,7 @@ public class PrepareEnableKerberosServerAction extends PrepareKerberosIdentities
   }
 
   @Override
-  protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal, KerberosOperationHandler operationHandler, Map<String, String> kerberosConfiguration, Map<String, Object> requestSharedDataContext) throws AmbariException {
+  protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal, KerberosOperationHandler operationHandler, Map<String, String> kerberosConfiguration, boolean includedInFilter, Map<String, Object> requestSharedDataContext) throws AmbariException {
     throw new UnsupportedOperationException();
   }
 }
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/serveraction/kerberos/KerberosServerActionTest.java b/ambari-server/src/test/java/org/apache/ambari/server/serveraction/kerberos/KerberosServerActionTest.java
index 27f30d8..d2f1aaf 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/serveraction/kerberos/KerberosServerActionTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/serveraction/kerberos/KerberosServerActionTest.java
@@ -96,6 +96,7 @@ public class KerberosServerActionTest extends EasyMockSupport {
       protected CommandReport processIdentity(ResolvedKerberosPrincipal resolvedPrincipal,
                                               KerberosOperationHandler operationHandler,
                                               Map<String, String> kerberosConfiguration,
+                                              boolean includedInFilter,
                                               Map<String, Object> requestSharedDataContext)
           throws AmbariException {
         Assert.assertNotNull(requestSharedDataContext);
diff --git a/ambari-web/app/controllers/main/host/details.js b/ambari-web/app/controllers/main/host/details.js
index 8a445dd..fe08664 100644
--- a/ambari-web/app/controllers/main/host/details.js
+++ b/ambari-web/app/controllers/main/host/details.js
@@ -3142,7 +3142,7 @@ App.MainHostDetailsController = Em.Controller.extend(App.SupportClientConfigsDow
         "RequestBodyInfo": {
           "RequestInfo": {
             context: Em.I18n.t('hosts.host.recover.regenerateKeytabs.context'),
-            query: "regenerate_keytabs=all&regenerate_hosts=" + hostName + "&ignore_config_updates=true",
+            query: "regenerate_keytabs=all&regenerate_hosts=" + hostName + "&config_update_policy=none",
           },
           "Body": {
             Clusters: {
diff --git a/ambari-web/app/utils/ajax/ajax.js b/ambari-web/app/utils/ajax/ajax.js
index 1308c21..1f92f7c 100644
--- a/ambari-web/app/utils/ajax/ajax.js
+++ b/ambari-web/app/utils/ajax/ajax.js
@@ -1946,7 +1946,7 @@ var urls = {
   },
 
   'admin.kerberos_security.regenerate_keytabs.service' : {
-    'real': '/clusters/{clusterName}?regenerate_keytabs=all&regenerate_components={serviceName}',
+    'real': '/clusters/{clusterName}?regenerate_keytabs=all&regenerate_components={serviceName}&config_update_policy=none',
     'mock': '',
     'type': 'PUT',
     'format': function (data) {
@@ -1961,7 +1961,7 @@ var urls = {
   },
 
   'admin.kerberos_security.regenerate_keytabs.host' : {
-    'real': '/clusters/{clusterName}?regenerate_keytabs=all&regenerate_hosts={hostName}',
+    'real': '/clusters/{clusterName}?regenerate_keytabs=all&regenerate_hosts={hostName}&config_update_policy=none',
     'mock': '',
     'type': 'PUT',
     'format': function (data) {