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 2015/10/28 15:36:27 UTC

ambari git commit: AMBARI-13551. When adding components to a Kerberized cluster, the set of hosts to create principals for should be limited to only the relevant set (rlevas)

Repository: ambari
Updated Branches:
  refs/heads/trunk 6254019a9 -> 72630f2ed


AMBARI-13551. When adding components to a Kerberized cluster, the set of hosts to create principals for should be limited to only the relevant set (rlevas)


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

Branch: refs/heads/trunk
Commit: 72630f2ed76b5be88cbcc98c4da0be780df85105
Parents: 6254019
Author: Robert Levas <rl...@hortonworks.com>
Authored: Wed Oct 28 10:36:15 2015 -0400
Committer: Robert Levas <rl...@hortonworks.com>
Committed: Wed Oct 28 10:36:20 2015 -0400

----------------------------------------------------------------------
 .../AmbariManagementControllerImpl.java         |   4 +-
 .../server/controller/KerberosHelper.java       |  39 ++++-
 .../server/controller/KerberosHelperImpl.java   | 106 +++++++++-----
 .../AbstractPrepareKerberosServerAction.java    |  13 ++
 .../kerberos/KerberosServerAction.java          |   5 +
 .../PrepareDisableKerberosServerAction.java     |   2 +-
 .../PrepareKerberosIdentitiesServerAction.java  |  12 +-
 .../server/controller/KerberosHelperTest.java   | 146 ++++++++++++++-----
 8 files changed, 236 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/72630f2e/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
index 152016a..3a04a90 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
@@ -2497,6 +2497,7 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
 
       if (!componentsToEnableKerberos.isEmpty()) {
         Map<String, Collection<String>> serviceFilter = new HashMap<String, Collection<String>>();
+        Set<String> hostFilter = new HashSet<String>();
 
         for (ServiceComponentHost scHost : componentsToEnableKerberos) {
           String serviceName = scHost.getServiceName();
@@ -2508,10 +2509,11 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
           }
 
           componentFilter.add(scHost.getServiceComponentName());
+          hostFilter.add(scHost.getHostName());
         }
 
         try {
-          kerberosHelper.ensureIdentities(cluster, serviceFilter, null, hostsToForceKerberosOperations, requestStages,
+          kerberosHelper.ensureIdentities(cluster, serviceFilter, hostFilter, null, hostsToForceKerberosOperations, requestStages,
               kerberosHelper.getManageIdentitiesDirective(requestProperties));
         } catch (KerberosOperationException e) {
           throw new IllegalArgumentException(e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/ambari/blob/72630f2e/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
index f87fb04..482756f 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
@@ -133,6 +133,9 @@ public interface KerberosHelper {
    *                                       relevant set of services and components - if null, no
    *                                       filter is relevant; if empty, the filter indicates no
    *                                       relevant services or components
+   * @param hostFilter                     a set of hostname indicating the set of hosts to process -
+   *                                       if null, no filter is relevant; if empty, the filter
+   *                                       indicates no relevant hosts
    * @param identityFilter                 a Collection of identity names indicating the relevant
    *                                       identities - if null, no filter is relevant; if empty,
    *                                       the filter indicates no relevant identities
@@ -155,8 +158,9 @@ public interface KerberosHelper {
    *                                               Kerberos-specific configuration details
    */
   RequestStageContainer ensureIdentities(Cluster cluster, Map<String, ? extends Collection<String>> serviceComponentFilter,
-                                         Collection<String> identityFilter, Set<String> hostsToForceKerberosOperations,
-                                         RequestStageContainer requestStageContainer, Boolean manageIdentities)
+                                         Set<String> hostFilter, Collection<String> identityFilter,
+                                         Set<String> hostsToForceKerberosOperations, RequestStageContainer requestStageContainer,
+                                         Boolean manageIdentities)
       throws AmbariException, KerberosOperationException;
 
   /**
@@ -177,6 +181,9 @@ public interface KerberosHelper {
    * @param serviceComponentFilter a Map of service names to component names indicating the relevant
    *                               set of services and components - if null, no filter is relevant;
    *                               if empty, the filter indicates no relevant services or components
+   * @param hostFilter             a set of hostname indicating the set of hosts to process -
+   *                               if null, no filter is relevant; if empty, the filter
+   *                               indicates no relevant hosts
    * @param identityFilter         a Collection of identity names indicating the relevant identities -
    *                               if null, no filter is relevant; if empty, the filter indicates no
    *                               relevant identities
@@ -193,8 +200,8 @@ public interface KerberosHelper {
    *                                               Kerberos-specific configuration details
    */
   RequestStageContainer deleteIdentities(Cluster cluster, Map<String, ? extends Collection<String>> serviceComponentFilter,
-                                         Collection<String> identityFilter, RequestStageContainer requestStageContainer,
-                                         Boolean manageIdentities)
+                                         Set<String> hostFilter, Collection<String> identityFilter,
+                                         RequestStageContainer requestStageContainer, Boolean manageIdentities)
       throws AmbariException, KerberosOperationException;
 
   /**
@@ -266,10 +273,30 @@ public interface KerberosHelper {
                            Map<String, Map<String, String>> kerberosConfigurations)
       throws AmbariException;
 
+  /**
+   * @param cluster                the cluster
+   * @param kerberosDescriptor     the current Kerberos descriptor
+   * @param serviceComponentFilter a Map of service names to component names indicating the
+   *                               relevant set of services and components - if null, no
+   *                               filter is relevant; if empty, the filter indicates no
+   *                               relevant services or components
+   * @param hostFilter             a set of hostname indicating the set of hosts to process -
+   *                               if null, no filter is relevant; if empty, the filter
+   *                               indicates no relevant hosts
+   * @param identityFilter         a Collection of identity names indicating the relevant
+   *                               identities - if null, no filter is relevant; if empty,
+   *                               the filter indicates no relevant identities
+   * @param shouldProcessCommand   a Command implementation to determine if the relevant component
+   *                               is in a state in which is should be process for the current
+   *                               Kerberos operation.
+   * @return a list of ServiceComponentHost instances and should be processed during the relevant
+   * Kerberos operation.
+   * @throws AmbariException
+   */
   List<ServiceComponentHost> getServiceComponentHostsToProcess(Cluster cluster,
                                                                KerberosDescriptor kerberosDescriptor,
                                                                Map<String, ? extends Collection<String>> serviceComponentFilter,
-                                                               Collection<String> identityFilter,
+                                                               Collection<String> hostFilter, Collection<String> identityFilter,
                                                                Command<Boolean, ServiceComponentHost> shouldProcessCommand)
       throws AmbariException;
 
@@ -423,10 +450,10 @@ public interface KerberosHelper {
   /**
    * Sets the previously stored KDC administrator credentials.
    *
+   * @param clusterName the name of the relevant cluster
    * @return a PrincipalKeyCredential or null, if the KDC administrator credentials have not be set or
    * have been removed
    * @throws AmbariException if an error occurs while retrieving the credentials
-   * @param clusterName
    */
   PrincipalKeyCredential getKDCAdministratorCredentials(String clusterName) throws AmbariException;
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/72630f2e/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 bf8c519..d162eec 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
@@ -209,10 +209,10 @@ public class KerberosHelperImpl implements KerberosHelper {
 
     if (securityType == SecurityType.KERBEROS) {
       LOG.info("Configuring Kerberos for realm {} on cluster, {}", kerberosDetails.getDefaultRealm(), cluster.getClusterName());
-      requestStageContainer = handle(cluster, kerberosDetails, null, null, null, requestStageContainer, new EnableKerberosHandler());
+      requestStageContainer = handle(cluster, kerberosDetails, null, null, null, null, requestStageContainer, new EnableKerberosHandler());
     } else if (securityType == SecurityType.NONE) {
       LOG.info("Disabling Kerberos from cluster, {}", cluster.getClusterName());
-      requestStageContainer = handle(cluster, kerberosDetails, null, null, null, requestStageContainer, new DisableKerberosHandler());
+      requestStageContainer = handle(cluster, kerberosDetails, null, null, null, null, requestStageContainer, new DisableKerberosHandler());
     } else {
       throw new AmbariException(String.format("Unexpected security type value: %s", securityType.name()));
     }
@@ -249,7 +249,7 @@ public class KerberosHelperImpl implements KerberosHelper {
 
               if (handler != null) {
                 requestStageContainer = handle(cluster, getKerberosDetails(cluster, manageIdentities),
-                    null, null, null, requestStageContainer, handler);
+                    null, null, null, null, requestStageContainer, handler);
               } else {
                 throw new AmbariException(String.format("Unexpected directive value: %s", value));
               }
@@ -269,19 +269,19 @@ public class KerberosHelperImpl implements KerberosHelper {
 
   @Override
   public RequestStageContainer ensureIdentities(Cluster cluster, Map<String, ? extends Collection<String>> serviceComponentFilter,
-                                                Collection<String> identityFilter, Set<String> hostsToForceKerberosOperations,
+                                                Set<String> hostFilter, Collection<String> identityFilter, Set<String> hostsToForceKerberosOperations,
                                                 RequestStageContainer requestStageContainer, Boolean manageIdentities)
       throws AmbariException, KerberosOperationException {
-    return handle(cluster, getKerberosDetails(cluster, manageIdentities), serviceComponentFilter, identityFilter,
+    return handle(cluster, getKerberosDetails(cluster, manageIdentities), serviceComponentFilter, hostFilter, identityFilter,
         hostsToForceKerberosOperations, requestStageContainer, new CreatePrincipalsAndKeytabsHandler(false, false));
   }
 
   @Override
   public RequestStageContainer deleteIdentities(Cluster cluster, Map<String, ? extends Collection<String>> serviceComponentFilter,
-                                                Collection<String> identityFilter, RequestStageContainer requestStageContainer,
-                                                Boolean manageIdentities)
+                                                Set<String> hostFilter, Collection<String> identityFilter,
+                                                RequestStageContainer requestStageContainer, Boolean manageIdentities)
       throws AmbariException, KerberosOperationException {
-    return handle(cluster, getKerberosDetails(cluster, manageIdentities), serviceComponentFilter, identityFilter, null,
+    return handle(cluster, getKerberosDetails(cluster, manageIdentities), serviceComponentFilter, hostFilter, identityFilter, null,
         requestStageContainer, new DeletePrincipalsAndKeytabsHandler());
   }
 
@@ -458,7 +458,7 @@ public class KerberosHelperImpl implements KerberosHelper {
   public List<ServiceComponentHost> getServiceComponentHostsToProcess(Cluster cluster,
                                                                       KerberosDescriptor kerberosDescriptor,
                                                                       Map<String, ? extends Collection<String>> serviceComponentFilter,
-                                                                      Collection<String> identityFilter,
+                                                                      Collection<String> hostFilter, Collection<String> identityFilter,
                                                                       Command<Boolean, ServiceComponentHost> shouldProcessCommand)
       throws AmbariException {
     List<ServiceComponentHost> serviceComponentHostsToProcess = new ArrayList<ServiceComponentHost>();
@@ -474,28 +474,31 @@ public class KerberosHelperImpl implements KerberosHelper {
         for (Host host : hosts) {
           String hostname = host.getHostName();
 
-          // Get a list of components on the current host
-          List<ServiceComponentHost> serviceComponentHosts = cluster.getServiceComponentHosts(hostname);
-
-          if ((serviceComponentHosts != null) && !serviceComponentHosts.isEmpty()) {
-
-            // Iterate over the components installed on the current host to get the service and
-            // component-level Kerberos descriptors in order to determine which principals,
-            // keytab files, and configurations need to be created or updated.
-            for (ServiceComponentHost sch : serviceComponentHosts) {
-              String serviceName = sch.getServiceName();
-              String componentName = sch.getServiceComponentName();
-
-              // If there is no filter or the filter contains the current service name...
-              if ((serviceComponentFilter == null) || serviceComponentFilter.containsKey(serviceName)) {
-                Collection<String> componentFilter = (serviceComponentFilter == null) ? null : serviceComponentFilter.get(serviceName);
-                KerberosServiceDescriptor serviceDescriptor = kerberosDescriptor.getService(serviceName);
-
-                if (serviceDescriptor != null) {
-                  // If there is no filter or the filter contains the current component name,
-                  // test to see if this component should be processed by querying the handler...
-                  if (((componentFilter == null) || componentFilter.contains(componentName)) && shouldProcessCommand.invoke(sch)) {
-                    serviceComponentHostsToProcess.add(sch);
+          // Filter hosts as needed....
+          if ((hostFilter == null) || hostFilter.contains(hostname)) {
+            // Get a list of components on the current host
+            List<ServiceComponentHost> serviceComponentHosts = cluster.getServiceComponentHosts(hostname);
+
+            if ((serviceComponentHosts != null) && !serviceComponentHosts.isEmpty()) {
+
+              // Iterate over the components installed on the current host to get the service and
+              // component-level Kerberos descriptors in order to determine which principals,
+              // keytab files, and configurations need to be created or updated.
+              for (ServiceComponentHost sch : serviceComponentHosts) {
+                String serviceName = sch.getServiceName();
+                String componentName = sch.getServiceComponentName();
+
+                // If there is no filter or the filter contains the current service name...
+                if ((serviceComponentFilter == null) || serviceComponentFilter.containsKey(serviceName)) {
+                  Collection<String> componentFilter = (serviceComponentFilter == null) ? null : serviceComponentFilter.get(serviceName);
+                  KerberosServiceDescriptor serviceDescriptor = kerberosDescriptor.getService(serviceName);
+
+                  if (serviceDescriptor != null) {
+                    // If there is no filter or the filter contains the current component name,
+                    // test to see if this component should be processed by querying the handler...
+                    if (((componentFilter == null) || componentFilter.contains(componentName)) && shouldProcessCommand.invoke(sch)) {
+                      serviceComponentHostsToProcess.add(sch);
+                    }
                   }
                 }
               }
@@ -1021,6 +1024,9 @@ public class KerberosHelperImpl implements KerberosHelper {
    * @param serviceComponentFilter         a Map of service names to component names indicating the relevant
    *                                       set of services and components - if null, no filter is relevant;
    *                                       if empty, the filter indicates no relevant services or components
+   * @param hostFilter                     a set of hostname indicating the set of hosts to process -
+   *                                       if null, no filter is relevant; if empty, the filter indicates no
+   *                                       relevant hosts
    * @param identityFilter                 a Collection of identity names indicating the relevant identities -
    *                                       if null, no filter is relevant; if empty, the filter indicates no
    *                                       relevant identities
@@ -1043,7 +1049,7 @@ public class KerberosHelperImpl implements KerberosHelper {
   private RequestStageContainer handle(Cluster cluster,
                                        KerberosDetails kerberosDetails,
                                        Map<String, ? extends Collection<String>> serviceComponentFilter,
-                                       Collection<String> identityFilter,
+                                       Set<String> hostFilter, Collection<String> identityFilter,
                                        Set<String> hostsToForceKerberosOperations,
                                        RequestStageContainer requestStageContainer,
                                        final Handler handler)
@@ -1056,6 +1062,7 @@ public class KerberosHelperImpl implements KerberosHelper {
         cluster,
         kerberosDescriptor,
         serviceComponentFilter,
+        hostFilter,
         identityFilter,
         new Command<Boolean, ServiceComponentHost>() {
           @Override
@@ -1122,7 +1129,7 @@ public class KerberosHelperImpl implements KerberosHelper {
     // Use the handler implementation to setup the relevant stages.
     handler.createStages(cluster, clusterHostInfoJson,
         hostParamsJson, event, roleCommandOrder, kerberosDetails, dataDirectory,
-        requestStageContainer, schToProcess, serviceComponentFilter, identityFilter,
+        requestStageContainer, schToProcess, serviceComponentFilter, hostFilter, identityFilter,
         hostsWithValidKerberosClient);
 
     // Add the finalize stage...
@@ -1366,7 +1373,7 @@ public class KerberosHelperImpl implements KerberosHelper {
           handler.createStages(cluster,
               clusterHostInfoJson, hostParamsJson, event, roleCommandOrder, kerberosDetails,
               dataDirectory, requestStageContainer, serviceComponentHostsToProcess,
-              Collections.<String, Collection<String>>emptyMap(), null, hostsWithValidKerberosClient);
+              Collections.<String, Collection<String>>emptyMap(), null, null, hostsWithValidKerberosClient);
 
 
           handler.addFinalizeOperationStage(cluster, clusterHostInfoJson, hostParamsJson, event,
@@ -1915,6 +1922,9 @@ public class KerberosHelperImpl implements KerberosHelper {
      * @param serviceComponentFilter a Map of service names to component names indicating the relevant
      *                               set of services and components - if null, no filter is relevant;
      *                               if empty, the filter indicates no relevant services or components
+     * @param hostFilter             a set of hostname indicating the set of hosts to process -
+     *                               if null, no filter is relevant; if empty, the filter indicates no
+     *                               relevant hosts
      * @param identityFilter         a Collection of identity names indicating the relevant identities -
      *                               if null, no filter is relevant; if empty, the filter indicates no
      *                               relevant identities
@@ -1928,8 +1938,10 @@ public class KerberosHelperImpl implements KerberosHelper {
                                KerberosDetails kerberosDetails, File dataDirectory,
                                RequestStageContainer requestStageContainer,
                                List<ServiceComponentHost> serviceComponentHosts,
-                               Map<String, ? extends Collection<String>> serviceComponentFilter, Collection<String> identityFilter, Set<String> hostsWithValidKerberosClient)
-        throws AmbariException;
+                               Map<String, ? extends Collection<String>> serviceComponentFilter,
+                               Set<String> hostFilter, Collection<String> identityFilter,
+                               Set<String> hostsWithValidKerberosClient)
+    throws AmbariException;
 
 
     public void addPrepareEnableKerberosOperationsStage(Cluster cluster, String clusterHostInfoJson,
@@ -2300,7 +2312,8 @@ public class KerberosHelperImpl implements KerberosHelper {
                              RoleCommandOrder roleCommandOrder, KerberosDetails kerberosDetails,
                              File dataDirectory, RequestStageContainer requestStageContainer,
                              List<ServiceComponentHost> serviceComponentHosts,
-                             Map<String, ? extends Collection<String>> serviceComponentFilter, Collection<String> identityFilter, Set<String> hostsWithValidKerberosClient)
+                             Map<String, ? extends Collection<String>> serviceComponentFilter,
+                             Set<String> hostFilter, Collection<String> identityFilter, Set<String> hostsWithValidKerberosClient)
         throws AmbariException {
       // If there are principals, keytabs, and configurations to process, setup the following sages:
       //  1) prepare identities
@@ -2329,6 +2342,9 @@ public class KerberosHelperImpl implements KerberosHelper {
       if (serviceComponentFilter != null) {
         commandParameters.put(KerberosServerAction.SERVICE_COMPONENT_FILTER, StageUtils.getGson().toJson(serviceComponentFilter));
       }
+      if (hostFilter != null) {
+        commandParameters.put(KerberosServerAction.HOST_FILTER, StageUtils.getGson().toJson(hostFilter));
+      }
       if (identityFilter != null) {
         commandParameters.put(KerberosServerAction.IDENTITY_FILTER, StageUtils.getGson().toJson(identityFilter));
       }
@@ -2412,7 +2428,7 @@ public class KerberosHelperImpl implements KerberosHelper {
                              RoleCommandOrder roleCommandOrder, KerberosDetails kerberosDetails,
                              File dataDirectory, RequestStageContainer requestStageContainer,
                              List<ServiceComponentHost> serviceComponentHosts,
-                             Map<String, ? extends Collection<String>> serviceComponentFilter, Collection<String> identityFilter, Set<String> hostsWithValidKerberosClient) throws AmbariException {
+                             Map<String, ? extends Collection<String>> serviceComponentFilter, Set<String> hostFilter, Collection<String> identityFilter, Set<String> hostsWithValidKerberosClient) throws AmbariException {
       //  1) revert configurations
 
       // If a RequestStageContainer does not already exist, create a new one...
@@ -2435,6 +2451,9 @@ public class KerberosHelperImpl implements KerberosHelper {
       if (serviceComponentFilter != null) {
         commandParameters.put(KerberosServerAction.SERVICE_COMPONENT_FILTER, StageUtils.getGson().toJson(serviceComponentFilter));
       }
+      if (hostFilter != null) {
+        commandParameters.put(KerberosServerAction.HOST_FILTER, StageUtils.getGson().toJson(hostFilter));
+      }
       if (identityFilter != null) {
         commandParameters.put(KerberosServerAction.IDENTITY_FILTER, StageUtils.getGson().toJson(identityFilter));
       }
@@ -2541,7 +2560,8 @@ public class KerberosHelperImpl implements KerberosHelper {
                              RoleCommandOrder roleCommandOrder, KerberosDetails kerberosDetails,
                              File dataDirectory, RequestStageContainer requestStageContainer,
                              List<ServiceComponentHost> serviceComponentHosts,
-                             Map<String, ? extends Collection<String>> serviceComponentFilter, Collection<String> identityFilter, Set<String> hostsWithValidKerberosClient)
+                             Map<String, ? extends Collection<String>> serviceComponentFilter,
+                             Set<String> hostFilter, Collection<String> identityFilter, Set<String> hostsWithValidKerberosClient)
         throws AmbariException {
       // If there are principals and keytabs to process, setup the following sages:
       //  1) prepare identities
@@ -2569,6 +2589,9 @@ public class KerberosHelperImpl implements KerberosHelper {
       if (serviceComponentFilter != null) {
         commandParameters.put(KerberosServerAction.SERVICE_COMPONENT_FILTER, StageUtils.getGson().toJson(serviceComponentFilter));
       }
+      if (hostFilter != null) {
+        commandParameters.put(KerberosServerAction.HOST_FILTER, StageUtils.getGson().toJson(hostFilter));
+      }
       if (identityFilter != null) {
         commandParameters.put(KerberosServerAction.IDENTITY_FILTER, StageUtils.getGson().toJson(identityFilter));
       }
@@ -2654,7 +2677,7 @@ public class KerberosHelperImpl implements KerberosHelper {
                              RoleCommandOrder roleCommandOrder, KerberosDetails kerberosDetails,
                              File dataDirectory, RequestStageContainer requestStageContainer,
                              List<ServiceComponentHost> serviceComponentHosts,
-                             Map<String, ? extends Collection<String>> serviceComponentFilter, Collection<String> identityFilter, Set<String> hostsWithValidKerberosClient)
+                             Map<String, ? extends Collection<String>> serviceComponentFilter, Set<String> hostFilter, Collection<String> identityFilter, Set<String> hostsWithValidKerberosClient)
         throws AmbariException {
 
       // If a RequestStageContainer does not already exist, create a new one...
@@ -2681,6 +2704,9 @@ public class KerberosHelperImpl implements KerberosHelper {
         if (serviceComponentFilter != null) {
           commandParameters.put(KerberosServerAction.SERVICE_COMPONENT_FILTER, StageUtils.getGson().toJson(serviceComponentFilter));
         }
+        if (hostFilter != null) {
+          commandParameters.put(KerberosServerAction.HOST_FILTER, StageUtils.getGson().toJson(hostFilter));
+        }
         if (identityFilter != null) {
           commandParameters.put(KerberosServerAction.IDENTITY_FILTER, StageUtils.getGson().toJson(identityFilter));
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/72630f2e/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/AbstractPrepareKerberosServerAction.java
index 479a054..359e651 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
@@ -41,6 +41,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 public abstract class AbstractPrepareKerberosServerAction extends KerberosServerAction {
   private final static Logger LOG = LoggerFactory.getLogger(AbstractPrepareKerberosServerAction.class);
@@ -178,6 +179,18 @@ public abstract class AbstractPrepareKerberosServerAction extends KerberosServer
     }
   }
 
+  protected Set<String> getHostFilter() {
+    String serializedValue = getCommandParameterValue(HOST_FILTER);
+
+    if(serializedValue != null) {
+      Type type = new TypeToken<Set<String>>() {}.getType();
+      return StageUtils.getGson().fromJson(serializedValue, type);
+    }
+    else {
+      return null;
+    }
+  }
+
   protected Collection<String> getIdentityFilter() {
     String serializedValue = getCommandParameterValue(IDENTITY_FILTER);
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/72630f2e/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/KerberosServerAction.java
----------------------------------------------------------------------
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 901a80f..90d9414 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
@@ -71,6 +71,11 @@ public abstract class KerberosServerAction extends AbstractServerAction {
   public static final String SERVICE_COMPONENT_FILTER = "service_component_filter";
 
   /**
+   * A (command parameter) property name used to hold the (serialized) host filter list.
+   */
+  public static final String HOST_FILTER = "host_filter";
+
+  /**
    * A (command parameter) property name used to hold the (serialized) identity filter list.
    */
   public static final String IDENTITY_FILTER = "identity_filter";

http://git-wip-us.apache.org/repos/asf/ambari/blob/72630f2e/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareDisableKerberosServerAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareDisableKerberosServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareDisableKerberosServerAction.java
index 4315f78..8ab04ff 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareDisableKerberosServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareDisableKerberosServerAction.java
@@ -91,7 +91,7 @@ public class PrepareDisableKerberosServerAction extends AbstractPrepareKerberosS
     List<ServiceComponentHost> schToProcess = kerberosHelper.getServiceComponentHostsToProcess(cluster,
         kerberosDescriptor,
         getServiceComponentFilter(),
-        identityFilter,
+        null, identityFilter,
         new KerberosHelper.Command<Boolean, ServiceComponentHost>() {
           @Override
           public Boolean invoke(ServiceComponentHost sch) throws AmbariException {

http://git-wip-us.apache.org/repos/asf/ambari/blob/72630f2e/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareKerberosIdentitiesServerAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareKerberosIdentitiesServerAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareKerberosIdentitiesServerAction.java
index 5f067ec..fba3eea 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareKerberosIdentitiesServerAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareKerberosIdentitiesServerAction.java
@@ -35,6 +35,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
 /**
@@ -95,7 +96,7 @@ public class PrepareKerberosIdentitiesServerAction extends AbstractPrepareKerber
 
     processServiceComponentHosts(cluster, kerberosDescriptor, schToProcess, identityFilter, dataDirectory, kerberosConfigurations);
 
-    if("true".equalsIgnoreCase(getCommandParameterValue(commandParameters, UPDATE_CONFIGURATIONS))) {
+    if ("true".equalsIgnoreCase(getCommandParameterValue(commandParameters, UPDATE_CONFIGURATIONS))) {
       processAuthToLocalRules(cluster, kerberosDescriptor, schToProcess, kerberosConfigurations, getDefaultRealm(commandParameters));
       processConfigurationChanges(dataDirectory, kerberosConfigurations);
     }
@@ -113,7 +114,7 @@ public class PrepareKerberosIdentitiesServerAction extends AbstractPrepareKerber
   }
 
   /**
-   * Calls {@link KerberosHelper#getServiceComponentHostsToProcess(Cluster, KerberosDescriptor, Map, Collection, KerberosHelper.Command)}
+   * Calls {@link KerberosHelper#getServiceComponentHostsToProcess(Cluster, KerberosDescriptor, Map, Collection, Collection, KerberosHelper.Command)}
    * with no filter on ServiceComponentHosts
    * <p/>
    * The <code>shouldProcessCommand</code> implementation passed to KerberosHelper#getServiceComponentHostsToProcess
@@ -121,10 +122,9 @@ public class PrepareKerberosIdentitiesServerAction extends AbstractPrepareKerber
    *
    * @param cluster            the cluster
    * @param kerberosDescriptor the current Kerberos descriptor
-   * @param identityFilter     a list of identities to include, or all if null
-   * @return the list of ServiceComponentHosts to process
+   * @param identityFilter     a list of identities to include, or all if null  @return the list of ServiceComponentHosts to process
    * @throws AmbariException
-   * @see KerberosHelper#getServiceComponentHostsToProcess(Cluster, KerberosDescriptor, Map, Collection, KerberosHelper.Command)
+   * @see KerberosHelper#getServiceComponentHostsToProcess(Cluster, KerberosDescriptor, Map, Collection, Collection, KerberosHelper.Command)
    */
   protected List<ServiceComponentHost> getServiceComponentHostsToProcess(Cluster cluster,
                                                                          KerberosDescriptor kerberosDescriptor,
@@ -133,7 +133,7 @@ public class PrepareKerberosIdentitiesServerAction extends AbstractPrepareKerber
     return kerberosHelper.getServiceComponentHostsToProcess(cluster,
         kerberosDescriptor,
         getServiceComponentFilter(),
-        identityFilter,
+        getHostFilter(), identityFilter,
         new KerberosHelper.Command<Boolean, ServiceComponentHost>() {
           @Override
           public Boolean invoke(ServiceComponentHost sch) throws AmbariException {

http://git-wip-us.apache.org/repos/asf/ambari/blob/72630f2e/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 d3f54e5..7a4f3e9 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
@@ -323,13 +323,13 @@ public class KerberosHelperTest extends EasyMockSupport {
 
   @Test
   public void testEnsureIdentities() throws Exception {
-    testEnsureIdentities(new PrincipalKeyCredential("principal", "password"));
+    testEnsureIdentities(new PrincipalKeyCredential("principal", "password"), null);
   }
 
   @Test(expected = KerberosMissingAdminCredentialsException.class)
   public void testEnsureIdentitiesMissingCredentials() throws Exception {
     try {
-      testEnsureIdentities(null);
+      testEnsureIdentities(null, null);
     } catch (IllegalArgumentException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing KDC administrator credentials"));
       throw e;
@@ -339,7 +339,7 @@ public class KerberosHelperTest extends EasyMockSupport {
   @Test(expected = KerberosMissingAdminCredentialsException.class)
   public void testEnsureIdentitiesInvalidCredentials() throws Exception {
     try {
-      testEnsureIdentities(new PrincipalKeyCredential("invalid_principal", "password"));
+      testEnsureIdentities(new PrincipalKeyCredential("invalid_principal", "password"), null);
     } catch (IllegalArgumentException e) {
       Assert.assertTrue(e.getMessage().startsWith("Invalid KDC administrator credentials"));
       throw e;
@@ -347,6 +347,11 @@ public class KerberosHelperTest extends EasyMockSupport {
   }
 
   @Test
+  public void testEnsureIdentities_FilteredHosts() throws Exception {
+    testEnsureIdentities(new PrincipalKeyCredential("principal", "password"), Collections.singleton("hostA"));
+  }
+
+  @Test
   public void testDeleteIdentities() throws Exception {
     testDeleteIdentities(new PrincipalKeyCredential("principal", "password"));
   }
@@ -2019,14 +2024,26 @@ public class KerberosHelperTest extends EasyMockSupport {
     expect(metaInfo.getKerberosDescriptor("HDP", "2.2")).andReturn(kerberosDescriptor).once();
   }
 
-  private void testEnsureIdentities(final PrincipalKeyCredential PrincipalKeyCredential) throws Exception {
+  private void testEnsureIdentities(final PrincipalKeyCredential PrincipalKeyCredential, Set<String> filteredHosts) throws Exception {
     KerberosHelper kerberosHelper = injector.getInstance(KerberosHelper.class);
 
-    final ServiceComponentHost schKerberosClient = createMock(ServiceComponentHost.class);
-    expect(schKerberosClient.getServiceName()).andReturn(Service.Type.KERBEROS.name()).anyTimes();
-    expect(schKerberosClient.getServiceComponentName()).andReturn(Role.KERBEROS_CLIENT.name()).anyTimes();
-    expect(schKerberosClient.getHostName()).andReturn("hostA").anyTimes();
-    expect(schKerberosClient.getState()).andReturn(State.INSTALLED).anyTimes();
+    final ServiceComponentHost schKerberosClientA = createMock(ServiceComponentHost.class);
+    expect(schKerberosClientA.getServiceName()).andReturn(Service.Type.KERBEROS.name()).anyTimes();
+    expect(schKerberosClientA.getServiceComponentName()).andReturn(Role.KERBEROS_CLIENT.name()).anyTimes();
+    expect(schKerberosClientA.getHostName()).andReturn("hostA").anyTimes();
+    expect(schKerberosClientA.getState()).andReturn(State.INSTALLED).anyTimes();
+
+    final ServiceComponentHost schKerberosClientB = createMock(ServiceComponentHost.class);
+    expect(schKerberosClientB.getServiceName()).andReturn(Service.Type.KERBEROS.name()).anyTimes();
+    expect(schKerberosClientB.getServiceComponentName()).andReturn(Role.KERBEROS_CLIENT.name()).anyTimes();
+    expect(schKerberosClientB.getHostName()).andReturn("hostB").anyTimes();
+    expect(schKerberosClientB.getState()).andReturn(State.INSTALLED).anyTimes();
+
+    final ServiceComponentHost schKerberosClientC = createMock(ServiceComponentHost.class);
+    expect(schKerberosClientC.getServiceName()).andReturn(Service.Type.KERBEROS.name()).anyTimes();
+    expect(schKerberosClientC.getServiceComponentName()).andReturn(Role.KERBEROS_CLIENT.name()).anyTimes();
+    expect(schKerberosClientC.getHostName()).andReturn("hostC").anyTimes();
+    expect(schKerberosClientC.getState()).andReturn(State.INSTALLED).anyTimes();
 
     final ServiceComponentHost sch1A = createMock(ServiceComponentHost.class);
     expect(sch1A.getServiceName()).andReturn("SERVICE1").anyTimes();
@@ -2067,7 +2084,15 @@ public class KerberosHelperTest extends EasyMockSupport {
 
     final ServiceComponent serviceComponentKerberosClient = createNiceMock(ServiceComponent.class);
     expect(serviceComponentKerberosClient.getName()).andReturn(Role.KERBEROS_CLIENT.name()).anyTimes();
-    expect(serviceComponentKerberosClient.getServiceComponentHosts()).andReturn(Collections.singletonMap("hostA", schKerberosClient)).anyTimes();
+    expect(serviceComponentKerberosClient.getServiceComponentHosts()).andReturn(
+        new HashMap<String, ServiceComponentHost>() {
+          {
+            put("hostA", schKerberosClientA);
+            put("hostB", schKerberosClientB);
+            put("hostC", schKerberosClientC);
+          }
+        }
+    ).anyTimes();
 
     final Service serviceKerberos = createStrictMock(Service.class);
     expect(serviceKerberos.getName()).andReturn(Service.Type.KERBEROS.name()).anyTimes();
@@ -2099,11 +2124,13 @@ public class KerberosHelperTest extends EasyMockSupport {
     final Config krb5ConfConfig = createMock(Config.class);
     expect(krb5ConfConfig.getProperties()).andReturn(krb5ConfProperties).anyTimes();
 
-    final Cluster cluster = createNiceMock(Cluster.class);
+    final Cluster cluster = createMock(Cluster.class);
     expect(cluster.getHosts()).andReturn(Arrays.asList(hostA, hostB, hostC)).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.getClusterId()).andReturn(1L).anyTimes();
+    expect(cluster.getSecurityType()).andReturn(SecurityType.KERBEROS).anyTimes();
     expect(cluster.getServices())
         .andReturn(new HashMap<String, Service>() {
           {
@@ -2113,28 +2140,48 @@ public class KerberosHelperTest extends EasyMockSupport {
           }
         })
         .anyTimes();
-    expect(cluster.getServiceComponentHosts("hostA"))
-        .andReturn(new ArrayList<ServiceComponentHost>() {
-          {
-            add(sch1A);
-            add(sch2);
-            add(sch3);
-            add(schKerberosClient);
-          }
-        })
-        .once();
-    expect(cluster.getServiceComponentHosts("hostB"))
-        .andReturn(new ArrayList<ServiceComponentHost>() {
-          {
-            add(sch1B);
-            add(schKerberosClient);
-          }
-        })
-        .once();
-    expect(cluster.getServiceComponentHosts("hostC"))
+
+    if ((filteredHosts == null) || filteredHosts.contains("hostA")) {
+      expect(cluster.getServiceComponentHosts("hostA"))
+          .andReturn(new ArrayList<ServiceComponentHost>() {
+            {
+              add(sch1A);
+              add(sch2);
+              add(sch3);
+              add(schKerberosClientA);
+            }
+          })
+          .once();
+    }
+
+    if ((filteredHosts == null) || filteredHosts.contains("hostB")) {
+      expect(cluster.getServiceComponentHosts("hostB"))
+          .andReturn(new ArrayList<ServiceComponentHost>() {
+            {
+              add(sch1B);
+              add(schKerberosClientB);
+            }
+          })
+          .once();
+    }
+
+    if ((filteredHosts == null) || filteredHosts.contains("hostC")) {
+      expect(cluster.getServiceComponentHosts("hostC"))
+          .andReturn(new ArrayList<ServiceComponentHost>() {
+            {
+              add(sch1C);
+              add(schKerberosClientC);
+            }
+          })
+          .once();
+    }
+
+      expect(cluster.getServiceComponentHosts("KERBEROS", "KERBEROS_CLIENT"))
         .andReturn(new ArrayList<ServiceComponentHost>() {
           {
-            add(sch1C);
+            add(schKerberosClientA);
+            add(schKerberosClientB);
+            add(schKerberosClientC);
           }
         })
         .once();
@@ -2142,6 +2189,23 @@ public class KerberosHelperTest extends EasyMockSupport {
         .andReturn(new StackId("HDP", "2.2"))
         .anyTimes();
 
+    final Clusters clusters = injector.getInstance(Clusters.class);
+    if ((filteredHosts == null) || filteredHosts.contains("hostA")) {
+      expect(clusters.getHost("hostA"))
+          .andReturn(hostA)
+          .once();
+    }
+    if ((filteredHosts == null) || filteredHosts.contains("hostB")) {
+      expect(clusters.getHost("hostB"))
+          .andReturn(hostB)
+          .once();
+    }
+    if ((filteredHosts == null) || filteredHosts.contains("hostC")) {
+      expect(clusters.getHost("hostC"))
+          .andReturn(hostC)
+          .once();
+    }
+
     final AmbariManagementController ambariManagementController = injector.getInstance(AmbariManagementController.class);
     expect(ambariManagementController.findConfigurationTagsWithOverrides(cluster, null))
         .andReturn(Collections.<String, Map<String, String>>emptyMap())
@@ -2191,10 +2255,18 @@ public class KerberosHelperTest extends EasyMockSupport {
     final KerberosServiceDescriptor serviceDescriptor3 = createMock(KerberosServiceDescriptor.class);
 
     final KerberosDescriptor kerberosDescriptor = createStrictMock(KerberosDescriptor.class);
-    expect(kerberosDescriptor.getService("SERVICE1")).andReturn(serviceDescriptor1).times(1);
-    expect(kerberosDescriptor.getService("SERVICE3")).andReturn(serviceDescriptor3).times(1);
-    expect(kerberosDescriptor.getService("SERVICE1")).andReturn(serviceDescriptor1).times(1);
-    expect(kerberosDescriptor.getService("SERVICE1")).andReturn(serviceDescriptor1).times(1);
+    if ((filteredHosts == null) || filteredHosts.contains("hostA")) {
+      expect(kerberosDescriptor.getService("SERVICE1")).andReturn(serviceDescriptor1).times(1);
+      expect(kerberosDescriptor.getService("SERVICE3")).andReturn(serviceDescriptor3).times(1);
+    }
+
+    if ((filteredHosts == null) || filteredHosts.contains("hostB")) {
+      expect(kerberosDescriptor.getService("SERVICE1")).andReturn(serviceDescriptor1).times(1);
+    }
+
+    if ((filteredHosts == null) || filteredHosts.contains("hostC")) {
+      expect(kerberosDescriptor.getService("SERVICE1")).andReturn(serviceDescriptor1).times(1);
+    }
 
     setupGetDescriptorFromCluster(kerberosDescriptor);
 
@@ -2259,7 +2331,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     credentialStoreService.setCredential(cluster.getClusterName(), KerberosHelper.KDC_ADMINISTRATOR_CREDENTIAL_ALIAS,
         PrincipalKeyCredential, CredentialStoreType.TEMPORARY);
 
-    kerberosHelper.ensureIdentities(cluster, serviceComponentFilter, identityFilter, null, requestStageContainer, true);
+    kerberosHelper.ensureIdentities(cluster, serviceComponentFilter, filteredHosts, identityFilter, null, requestStageContainer, true);
 
     verifyAll();
   }
@@ -2463,7 +2535,7 @@ public class KerberosHelperTest extends EasyMockSupport {
     credentialStoreService.setCredential(cluster.getClusterName(), KerberosHelper.KDC_ADMINISTRATOR_CREDENTIAL_ALIAS,
         PrincipalKeyCredential, CredentialStoreType.TEMPORARY);
 
-    kerberosHelper.deleteIdentities(cluster, serviceComponentFilter, identityFilter, requestStageContainer, true);
+    kerberosHelper.deleteIdentities(cluster, serviceComponentFilter, null, identityFilter, requestStageContainer, true);
 
     verifyAll();
   }