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 2017/09/01 16:31:46 UTC

[3/3] ambari git commit: AMBARI-21809. Pre-configure services during stack upgrade if Kerberos is enabled to reduce number of core service restarts when services are added (rlevas)

AMBARI-21809. Pre-configure services during stack upgrade if Kerberos is enabled to reduce number of core service restarts when services are added (rlevas)


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

Branch: refs/heads/trunk
Commit: 5230d93545c30ab06d49f9cbac25974b1462f914
Parents: e20c064
Author: Robert Levas <rl...@hortonworks.com>
Authored: Fri Sep 1 12:31:23 2017 -0400
Committer: Robert Levas <rl...@hortonworks.com>
Committed: Fri Sep 1 12:31:23 2017 -0400

----------------------------------------------------------------------
 .../controller/DeleteIdentityHandler.java       |   2 +-
 .../server/controller/KerberosHelper.java       |  65 +-
 .../server/controller/KerberosHelperImpl.java   | 142 ++--
 .../PrepareDisableKerberosServerAction.java     |  33 +-
 .../PrepareEnableKerberosServerAction.java      |   3 +-
 .../PrepareKerberosIdentitiesServerAction.java  |   5 +-
 .../upgrades/PreconfigureKerberosAction.java    | 573 +++++++++++++++
 .../HDP/2.5/upgrades/nonrolling-upgrade-2.6.xml |  12 +
 .../stacks/HDP/2.5/upgrades/upgrade-2.6.xml     |  12 +
 .../PreconfigureKerberosActionTest.java         | 596 +++++++++++++++
 .../PreconfigureActionTest_cluster_config.json  | 110 +++
 ...ureActionTest_kerberos_descriptor_stack.json | 713 ++++++++++++++++++
 ..._kerberos_descriptor_stack_preconfigure.json | 730 +++++++++++++++++++
 13 files changed, 2920 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/5230d935/ambari-server/src/main/java/org/apache/ambari/server/controller/DeleteIdentityHandler.java
----------------------------------------------------------------------
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 978b329..a7b9d80 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
@@ -246,7 +246,7 @@ class DeleteIdentityHandler {
      * The service configuration is needed because principal names may contain placeholder variables which are replaced based on the service configuration.
      */
     private Map<String, Map<String, String>> calculateConfig(KerberosDescriptor kerberosDescriptor, Set<String> serviceNames) throws AmbariException {
-      Map<String, Map<String, String>> actualConfig = getKerberosHelper().calculateConfigurations(getCluster(), null, kerberosDescriptor.getProperties());
+      Map<String, Map<String, String>> actualConfig = getKerberosHelper().calculateConfigurations(getCluster(), null, kerberosDescriptor, false, false);
       extendWithDeletedConfigOfService(actualConfig, serviceNames);
       return actualConfig;
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/5230d935/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 9bdb377..bb360b5 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
@@ -36,6 +36,7 @@ import org.apache.ambari.server.serveraction.kerberos.KerberosOperationException
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.SecurityType;
 import org.apache.ambari.server.state.ServiceComponentHost;
+import org.apache.ambari.server.state.StackId;
 import org.apache.ambari.server.state.kerberos.KerberosConfigurationDescriptor;
 import org.apache.ambari.server.state.kerberos.KerberosDescriptor;
 import org.apache.ambari.server.state.kerberos.KerberosIdentityDescriptor;
@@ -510,6 +511,30 @@ public interface KerberosHelper {
       throws AmbariException;
 
   /**
+   * Gets the Kerberos descriptor for the requested stack.
+   * <p>
+   * One of the following Kerberos descriptors will be returned:
+   * <dl>
+   * <dt>{@link KerberosDescriptorType#STACK}</dt>
+   * <dd>A Kerberos descriptor built using data from the current stack definition, only</dd>
+   * <dt>{@link KerberosDescriptorType#USER}</dt>
+   * <dd>A Kerberos descriptor built using user-specified data stored as an artifact of the cluster, only</dd>
+   * <dt>{@link KerberosDescriptorType#COMPOSITE}</dt>
+   * <dd>A Kerberos descriptor built using data from the current stack definition with user-specified data stored as an artifact of the cluster applied
+   * - see {@link #getKerberosDescriptor(Cluster, boolean)}</dd>
+   * </dl>
+   *
+   * @param kerberosDescriptorType  the type of Kerberos descriptor to retrieve - see {@link KerberosDescriptorType}
+   * @param cluster                 the relevant Cluster
+   * @param stackId                 the relevant stack id, used for <code>COMPOSITE</code> or <code>STACK</code> Kerberos descriptor requests
+   * @param includePreconfigureData <code>true</code> to include the preconfigure data; <code>false</code> otherwise
+   * @return a Kerberos descriptor
+   * @throws AmbariException
+   */
+  KerberosDescriptor getKerberosDescriptor(KerberosDescriptorType kerberosDescriptorType, Cluster cluster, StackId stackId, boolean includePreconfigureData)
+      throws AmbariException;
+
+  /**
    * Merges configurations from a Map of configuration updates into a main configurations Map.
    * <p>
    * Each property in the updates Map is processed to replace variables using the replacement Map,
@@ -583,12 +608,16 @@ public interface KerberosHelper {
    *
    * @param cluster                      the relevant Cluster
    * @param hostname                     the relevant hostname
-   * @param kerberosDescriptorProperties a map of general Kerberos descriptor properties
+   * @param kerberosDescriptor a map of general Kerberos descriptor properties
+   * @param includePreconfigureData <code>true</code> to include the preconfigure data; otherwise false
+   * @param calculateClusterHostInfo
    * @return a Map of calculated configuration types
    * @throws AmbariException
    */
   Map<String, Map<String, String>> calculateConfigurations(Cluster cluster, String hostname,
-                                                           Map<String, String> kerberosDescriptorProperties)
+                                                           KerberosDescriptor kerberosDescriptor,
+                                                           boolean includePreconfigureData,
+                                                           boolean calculateClusterHostInfo)
       throws AmbariException;
 
   /**
@@ -709,6 +738,38 @@ public interface KerberosHelper {
   PrincipalKeyCredential getKDCAdministratorCredentials(String clusterName) throws AmbariException;
 
   /**
+   * Translates a collection of configuration specifications (<code>config-type/property-name</code>)
+   * to a map of configuration types to a set of property names.
+   * <p>
+   * For example:
+   * <ul>
+   * <li>config-type1/property-name1</li>
+   * <li>config-type1/property-name2</li>
+   * <li>config-type2/property-name3</li>
+   * </ul>
+   * Becomes
+   * <ul>
+   * <li>
+   * config-type
+   * <ul>
+   * <li>property-name1</li>
+   * <li>property-name2</li>
+   * </ul>
+   * </li>
+   * <li>
+   * config-type2
+   * <ul>
+   * <li>property-name3</li>
+   * </ul>
+   * </li>
+   * </ul>
+   *
+   * @param configurationSpecifications a collection of configuration specifications (<code>config-type/property-name</code>)
+   * @return a map of configuration types to sets of property names
+   */
+  Map<String, Set<String>> translateConfigurationSpecifications(Collection<String> configurationSpecifications);
+
+  /**
    * Types of Kerberos descriptors related to where the data is stored.
    * <dl>
    * <dt>STACK</dt>

http://git-wip-us.apache.org/repos/asf/ambari/blob/5230d935/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 d86433f..013a063 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
@@ -1328,15 +1328,20 @@ public class KerberosHelperImpl implements KerberosHelper {
                                                   boolean includePreconfigureData)
       throws AmbariException {
 
-    KerberosDescriptor stackDescriptor = (kerberosDescriptorType == KerberosDescriptorType.STACK || kerberosDescriptorType == KerberosDescriptorType.COMPOSITE)
-        ? getKerberosDescriptorFromStack(cluster, includePreconfigureData)
-        : null;
+    // !!! FIXME in a per-service view, what does this become?
+    Set<StackId> stackIds = new HashSet<>();
 
-    KerberosDescriptor userDescriptor = (kerberosDescriptorType == KerberosDescriptorType.USER || kerberosDescriptorType == KerberosDescriptorType.COMPOSITE)
-        ? getKerberosDescriptorUpdates(cluster)
-        : null;
+    for (Service service : cluster.getServices().values()) {
+      stackIds.add(service.getDesiredStackId());
+    }
 
-    KerberosDescriptor kerberosDescriptor = combineKerberosDescriptors(stackDescriptor, userDescriptor);
+    if (1 != stackIds.size()) {
+      throw new AmbariException("Services are deployed from multiple stacks and cannot determine a unique one.");
+    }
+
+    StackId stackId = stackIds.iterator().next();
+
+    KerberosDescriptor kerberosDescriptor = getKerberosDescriptor(kerberosDescriptorType, cluster, stackId, includePreconfigureData);
 
     if (evaluateWhenClauses) {
       Set<String> services = new HashSet<>(cluster.getServices().keySet());
@@ -1348,7 +1353,7 @@ public class KerberosHelperImpl implements KerberosHelper {
       // Build the context needed to filter out Kerberos identities...
       // This includes the current set of configurations for the cluster and the set of installed services
       Map<String, Object> context = new HashMap<>();
-      context.put("configurations", calculateConfigurations(cluster, null, kerberosDescriptor.getProperties()));
+      context.put("configurations", calculateConfigurations(cluster, null, kerberosDescriptor, false, false));
       context.put("services", services);
 
       // Get the Kerberos identities that need to be pruned
@@ -1385,6 +1390,20 @@ public class KerberosHelperImpl implements KerberosHelper {
   }
 
   @Override
+  public KerberosDescriptor getKerberosDescriptor(KerberosDescriptorType kerberosDescriptorType, Cluster cluster,
+                                                  StackId stackId, boolean includePreconfigureData) throws AmbariException {
+    KerberosDescriptor stackDescriptor = (kerberosDescriptorType == KerberosDescriptorType.STACK || kerberosDescriptorType == KerberosDescriptorType.COMPOSITE)
+        ? getKerberosDescriptorFromStack(stackId, includePreconfigureData)
+        : null;
+
+    KerberosDescriptor userDescriptor = (kerberosDescriptorType == KerberosDescriptorType.USER || kerberosDescriptorType == KerberosDescriptorType.COMPOSITE)
+        ? getKerberosDescriptorUpdates(cluster)
+        : null;
+
+    return combineKerberosDescriptors(stackDescriptor, userDescriptor);
+  }
+
+  @Override
   public Map<String, Map<String, String>> mergeConfigurations(Map<String, Map<String, String>> configurations,
                                                               Map<String, KerberosConfigurationDescriptor> updates,
                                                               Map<String, Map<String, String>> replacements,
@@ -1503,18 +1522,20 @@ public class KerberosHelperImpl implements KerberosHelper {
             }
 
             // Append an entry to the action data file builder...
-            kerberosIdentityDataFileWriter.writeRecord(
-                hostname,
-                serviceName,
-                componentName,
-                principal,
-                principalType,
-                keytabFilePath,
-                keytabFileOwnerName,
-                keytabFileOwnerAccess,
-                keytabFileGroupName,
-                keytabFileGroupAccess,
-                (keytabIsCachable) ? "true" : "false");
+            if(kerberosIdentityDataFileWriter != null) {
+              kerberosIdentityDataFileWriter.writeRecord(
+                  hostname,
+                  serviceName,
+                  componentName,
+                  principal,
+                  principalType,
+                  keytabFilePath,
+                  keytabFileOwnerName,
+                  keytabFileOwnerAccess,
+                  keytabFileGroupName,
+                  keytabFileGroupAccess,
+                  (keytabIsCachable) ? "true" : "false");
+            }
 
             // Add the principal-related configuration to the map of configurations
             mergeConfiguration(kerberosConfigurations, principalConfiguration, principal, null);
@@ -1533,11 +1554,22 @@ public class KerberosHelperImpl implements KerberosHelper {
 
   @Override
   public Map<String, Map<String, String>> calculateConfigurations(Cluster cluster, String hostname,
-                                                                  Map<String, String> kerberosDescriptorProperties)
+                                                                  KerberosDescriptor kerberosDescriptor,
+                                                                  boolean includePreconfigureData,
+                                                                  boolean calculateClusterHostInfo)
       throws AmbariException {
-    return addAdditionalConfigurations(cluster,
+
+
+    Map<String, Map<String, String>> calculatedConfigurations = addAdditionalConfigurations(
+        cluster,
         calculateExistingConfigurations(cluster, hostname),
-        hostname, kerberosDescriptorProperties);
+        hostname,
+        (kerberosDescriptor == null) ? null : kerberosDescriptor.getProperties());
+
+    if (includePreconfigureData) {
+      calculatedConfigurations = addConfigurationsForPreProcessedServices(calculatedConfigurations, cluster, kerberosDescriptor, calculateClusterHostInfo);
+    }
+    return calculatedConfigurations;
   }
 
   private Map<String, String> principalNames(Cluster cluster, Map<String, Map<String, String>> configuration) throws AmbariException {
@@ -1600,8 +1632,6 @@ public class KerberosHelperImpl implements KerberosHelper {
         KerberosDescriptor kerberosDescriptor = getKerberosDescriptor(cluster, false);
 
         if (kerberosDescriptor != null) {
-          Map<String, String> kerberosDescriptorProperties = kerberosDescriptor.getProperties();
-
           Set<String> existingServices = cluster.getServices().keySet();
 
           for (String hostname : hosts) {
@@ -1609,7 +1639,9 @@ public class KerberosHelperImpl implements KerberosHelper {
             // variables within the Kerberos descriptor data
             Map<String, Map<String, String>> configurations = calculateConfigurations(cluster,
                 hostname.equals(ambariServerHostname) ? null : hostname,
-                kerberosDescriptorProperties);
+                kerberosDescriptor,
+                false,
+                false);
 
             // Create the context to use for filtering Kerberos Identities based on the state of the cluster
             Map<String, Object> filterContext = new HashMap<>();
@@ -1755,6 +1787,37 @@ public class KerberosHelperImpl implements KerberosHelper {
     }
   }
 
+  @Override
+  public Map<String, Set<String>> translateConfigurationSpecifications(Collection<String> configurationSpecifications) {
+    Map<String, Set<String>> translation = null;
+
+    if (configurationSpecifications != null) {
+      translation = new HashMap<>();
+
+      for (String configurationSpecification : configurationSpecifications) {
+        Matcher m = KerberosDescriptor.AUTH_TO_LOCAL_PROPERTY_SPECIFICATION_PATTERN.matcher(configurationSpecification);
+
+        if (m.matches()) {
+          String configType = m.group(1);
+          String propertyName = m.group(2);
+
+          if (configType == null) {
+            configType = "";
+          }
+
+          Set<String> propertyNames = translation.get(configType);
+          if(propertyNames == null) {
+            propertyNames = new HashSet<>();
+            translation.put(configType, propertyNames);
+          }
+          propertyNames.add(propertyName);
+        }
+      }
+    }
+
+    return translation;
+  }
+
   /**
    * Creates the principal and cached keytab file for the specified identity, if it is determined to
    * be of the expected type - user (headless) or service.
@@ -2072,8 +2135,6 @@ public class KerberosHelperImpl implements KerberosHelper {
       KerberosDescriptor kerberosDescriptor = getKerberosDescriptor(cluster, false);
       KerberosIdentityDataFileWriter kerberosIdentityDataFileWriter = null;
 
-      Map<String, String> kerberosDescriptorProperties = kerberosDescriptor.getProperties();
-
       // This is needed to help determine which hosts to perform actions for and create tasks for.
       Set<String> hostsWithValidKerberosClient = getHostsWithValidKerberosClient(cluster);
 
@@ -2088,7 +2149,7 @@ public class KerberosHelperImpl implements KerberosHelper {
 
       // Calculate the current non-host-specific configurations. These will be used to replace
       // variables within the Kerberos descriptor data
-      Map<String, Map<String, String>> configurations = calculateConfigurations(cluster, null, kerberosDescriptorProperties);
+      Map<String, Map<String, String>> configurations = calculateConfigurations(cluster, null, kerberosDescriptor, false, false);
 
       String principal = variableReplacementHelper.replaceVariables("${kerberos-env/service_check_principal_name}@${realm}", configurations);
       String principalType = "user";
@@ -2839,31 +2900,14 @@ public class KerberosHelperImpl implements KerberosHelper {
   }
 
   /**
-   * Get the default Kerberos descriptor from the stack, which is the same as the value from
-   * <code>stacks/:stackName/versions/:version/artifacts/kerberos_descriptor</code>
+   * Get the default Kerberos descriptor from the specified stack.
    *
-   * @param cluster                 the cluster
+   * @param stackId                 the relevant stack ID
    * @param includePreconfigureData <code>true</code> to include the preconfigure data; otherwise false
    * @return a Kerberos Descriptor
    * @throws AmbariException if an error occurs while retrieving the Kerberos descriptor
    */
-  private KerberosDescriptor getKerberosDescriptorFromStack(Cluster cluster, boolean includePreconfigureData) throws AmbariException {
-    // !!! FIXME in a per-service view, what does this become?
-    Set<StackId> stackIds = new HashSet<>();
-
-    for (Service service : cluster.getServices().values()) {
-      stackIds.add(service.getDesiredStackId());
-    }
-
-    if (1 != stackIds.size()) {
-      throw new AmbariException("Services are deployed from multiple stacks and cannot determine a unique one.");
-    }
-
-    StackId stackId = stackIds.iterator().next();
-
-    // -------------------------------
-    // Get the default Kerberos descriptor from the stack, which is the same as the value from
-    // stacks/:stackName/versions/:version/artifacts/kerberos_descriptor
+  private KerberosDescriptor getKerberosDescriptorFromStack(StackId stackId, boolean includePreconfigureData) throws AmbariException {
     return ambariMetaInfo.getKerberosDescriptor(stackId.getStackName(), stackId.getStackVersion(), includePreconfigureData);
     // -------------------------------
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/5230d935/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 178d129..4e63f4a 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
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
-import java.util.regex.Matcher;
 
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.actionmanager.HostRoleStatus;
@@ -38,6 +37,7 @@ import org.apache.ambari.server.state.ServiceComponent;
 import org.apache.ambari.server.state.ServiceComponentHost;
 import org.apache.ambari.server.state.StackId;
 import org.apache.ambari.server.state.kerberos.KerberosDescriptor;
+import org.apache.commons.collections.CollectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -99,36 +99,31 @@ public class PrepareDisableKerberosServerAction extends AbstractPrepareKerberosS
       actionLog.writeStdOut(String.format("Processing %d components", schCount));
     }
 
-    Map<String, String> kerberosDescriptorProperties = kerberosDescriptor.getProperties();
     Set<String> services = cluster.getServices().keySet();
     boolean includeAmbariIdentity = "true".equalsIgnoreCase(getCommandParameterValue(commandParameters, KerberosServerAction.INCLUDE_AMBARI_IDENTITY));
     Map<String, Set<String>> propertiesToIgnore = new HashMap<>();
 
     // Calculate the current host-specific configurations. These will be used to replace
     // variables within the Kerberos descriptor data
-    Map<String, Map<String, String>> configurations = kerberosHelper.calculateConfigurations(cluster, null, kerberosDescriptorProperties);
+    Map<String, Map<String, String>> configurations = kerberosHelper.calculateConfigurations(cluster, null, kerberosDescriptor, false, false);
 
     processServiceComponentHosts(cluster, kerberosDescriptor, schToProcess, identityFilter, dataDirectory,
         configurations, kerberosConfigurations, includeAmbariIdentity, propertiesToIgnore);
 
     // Add auth-to-local configurations to the set of changes
-    Set<String> authToLocalProperties = kerberosDescriptor.getAllAuthToLocalProperties();
+    Map<String, Set<String>> authToLocalProperties = kerberosHelper.translateConfigurationSpecifications(kerberosDescriptor.getAllAuthToLocalProperties());
     if (authToLocalProperties != null) {
-      for (String authToLocalProperty : authToLocalProperties) {
-        Matcher m = KerberosDescriptor.AUTH_TO_LOCAL_PROPERTY_SPECIFICATION_PATTERN.matcher(authToLocalProperty);
-
-        if (m.matches()) {
-          String configType = m.group(1);
-          String propertyName = m.group(2);
-
-          if (configType == null) {
-            configType = "";
-          }
-
-          // Add existing auth_to_local configuration, if set
-          Map<String, String> configuration = kerberosConfigurations.get(configType);
-          if (configuration != null) {
-            configuration.put(propertyName, "DEFAULT");
+      for (Map.Entry<String, Set<String>> entry : authToLocalProperties.entrySet()) {
+        String configType = entry.getKey();
+        Set<String> propertyNames = entry.getValue();
+
+        if (!CollectionUtils.isEmpty(propertyNames)) {
+          for (String propertyName : propertyNames) {
+            // Add existing auth_to_local configuration, if set
+            Map<String, String> configuration = kerberosConfigurations.get(configType);
+            if (configuration != null) {
+              configuration.put(propertyName, "DEFAULT");
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/ambari/blob/5230d935/ambari-server/src/main/java/org/apache/ambari/server/serveraction/kerberos/PrepareEnableKerberosServerAction.java
----------------------------------------------------------------------
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 da83a74..e13f033 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
@@ -108,14 +108,13 @@ public class PrepareEnableKerberosServerAction extends PrepareKerberosIdentities
     }
 
     KerberosHelper kerberosHelper = getKerberosHelper();
-    Map<String, String> kerberosDescriptorProperties = kerberosDescriptor.getProperties();
     Map<String, Set<String>> propertiesToRemove = new HashMap<>();
     Map<String, Set<String>> propertiesToIgnore = new HashMap<>();
     Set<String> services = cluster.getServices().keySet();
 
     // Calculate the current host-specific configurations. These will be used to replace
     // variables within the Kerberos descriptor data
-    Map<String, Map<String, String>> configurations = kerberosHelper.calculateConfigurations(cluster, null, kerberosDescriptorProperties);
+    Map<String, Map<String, String>> configurations = kerberosHelper.calculateConfigurations(cluster, null, kerberosDescriptor, false, false);
 
     processServiceComponentHosts(cluster, kerberosDescriptor, schToProcess, identityFilter, dataDirectory,
         configurations, kerberosConfigurations, true, propertiesToIgnore);

http://git-wip-us.apache.org/repos/asf/ambari/blob/5230d935/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 581067f..00c82a5 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
@@ -84,7 +84,6 @@ public class PrepareKerberosIdentitiesServerAction extends AbstractPrepareKerber
     }
 
     KerberosHelper kerberosHelper = getKerberosHelper();
-    Map<String, String> kerberosDescriptorProperties = kerberosDescriptor.getProperties();
     Set<String> services = cluster.getServices().keySet();
     Map<String, Set<String>> propertiesToRemove = new HashMap<>();
     Map<String, Set<String>> propertiesToIgnore = new HashMap<>();
@@ -92,7 +91,7 @@ public class PrepareKerberosIdentitiesServerAction extends AbstractPrepareKerber
 
     // Calculate the current host-specific configurations. These will be used to replace
     // variables within the Kerberos descriptor data
-    Map<String, Map<String, String>> configurations = kerberosHelper.calculateConfigurations(cluster, null, kerberosDescriptorProperties);
+    Map<String, Map<String, String>> configurations = kerberosHelper.calculateConfigurations(cluster, null, kerberosDescriptor, false, false);
 
     processServiceComponentHosts(cluster, kerberosDescriptor, schToProcess, identityFilter, dataDirectory,
         configurations, kerberosConfigurations, includeAmbariIdentity, propertiesToIgnore);
@@ -101,7 +100,7 @@ public class PrepareKerberosIdentitiesServerAction extends AbstractPrepareKerber
         propertiesToIgnore, propertiesToRemove, true);
 
     if ("true".equalsIgnoreCase(getCommandParameterValue(commandParameters, UPDATE_CONFIGURATIONS))) {
-      Map<String, Map<String, String>> calculatedConfigurations = kerberosHelper.calculateConfigurations(cluster, null, kerberosDescriptor.getProperties());
+      Map<String, Map<String, String>> calculatedConfigurations = kerberosHelper.calculateConfigurations(cluster, null, kerberosDescriptor, false, false);
       processAuthToLocalRules(cluster, calculatedConfigurations, kerberosDescriptor, schToProcess, kerberosConfigurations, getDefaultRealm(commandParameters), false);
       processConfigurationChanges(dataDirectory, kerberosConfigurations, propertiesToRemove);
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/5230d935/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/PreconfigureKerberosAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/PreconfigureKerberosAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/PreconfigureKerberosAction.java
new file mode 100644
index 0000000..697f1d1
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/PreconfigureKerberosAction.java
@@ -0,0 +1,573 @@
+/*
+ * 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.upgrades;
+
+import static org.apache.ambari.server.controller.KerberosHelper.DEFAULT_REALM;
+import static org.apache.ambari.server.controller.KerberosHelper.KERBEROS_ENV;
+import static org.apache.ambari.server.controller.KerberosHelper.PRECONFIGURE_SERVICES;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.actionmanager.HostRoleStatus;
+import org.apache.ambari.server.agent.CommandReport;
+import org.apache.ambari.server.controller.AmbariManagementController;
+import org.apache.ambari.server.controller.KerberosHelper;
+import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
+import org.apache.ambari.server.serveraction.kerberos.PreconfigureServiceType;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.ConfigHelper;
+import org.apache.ambari.server.state.Host;
+import org.apache.ambari.server.state.SecurityType;
+import org.apache.ambari.server.state.Service;
+import org.apache.ambari.server.state.ServiceComponentHost;
+import org.apache.ambari.server.state.StackId;
+import org.apache.ambari.server.state.UpgradeContext;
+import org.apache.ambari.server.state.kerberos.AbstractKerberosDescriptorContainer;
+import org.apache.ambari.server.state.kerberos.KerberosComponentDescriptor;
+import org.apache.ambari.server.state.kerberos.KerberosConfigurationDescriptor;
+import org.apache.ambari.server.state.kerberos.KerberosDescriptor;
+import org.apache.ambari.server.state.kerberos.KerberosIdentityDescriptor;
+import org.apache.ambari.server.state.kerberos.KerberosServiceDescriptor;
+import org.apache.ambari.server.state.kerberos.VariableReplacementHelper;
+import org.apache.ambari.server.state.stack.upgrade.Direction;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang.StringUtils;
+
+import com.google.inject.Inject;
+
+/**
+ * PreconfigureKerberos updates existing service configurations with properties from service-level
+ * Kerberos descriptors, flagged for pre-configuring, during stack upgrades in order to prevent service
+ * restarts when the flagged services are installed.
+ */
+public class PreconfigureKerberosAction extends AbstractUpgradeServerAction {
+  static final String UPGRADE_DIRECTION_KEY = "upgrade_direction";
+
+  @Inject
+  private AmbariManagementController ambariManagementController;
+
+  @Inject
+  private KerberosHelper kerberosHelper;
+
+  @Inject
+  private ConfigHelper configHelper;
+
+  @Inject
+  private VariableReplacementHelper variableReplacementHelper;
+
+  @Override
+  public CommandReport execute(ConcurrentMap<String, Object> requestSharedDataContext) throws AmbariException, InterruptedException {
+    Map<String, String> commandParameters = getCommandParameters();
+    if (null == commandParameters || commandParameters.isEmpty()) {
+      return createCommandReport(0, HostRoleStatus.FAILED, "{}", "",
+          "Unable to change configuration values without command parameters");
+    }
+
+    if (!isDowngrade()) {
+      String clusterName = commandParameters.get("clusterName");
+      Cluster cluster = m_clusters.getCluster(clusterName);
+
+      if (cluster.getSecurityType() == SecurityType.KERBEROS) {
+        StackId stackId;
+
+        try {
+          stackId = getTargetStackId(cluster);
+        } catch (AmbariException e) {
+          return createCommandReport(0, HostRoleStatus.FAILED, "{}", "", e.getLocalizedMessage());
+        }
+
+        if (stackId == null) {
+          return createCommandReport(0, HostRoleStatus.FAILED, "{}", "",
+              "The target stack Id was not specified.");
+        }
+
+        KerberosDescriptor kerberosDescriptor = kerberosHelper.getKerberosDescriptor(KerberosHelper.KerberosDescriptorType.COMPOSITE, cluster, stackId, true);
+
+        // Calculate the current host-specific configurations. These will be used to replace
+        // variables within the Kerberos descriptor data
+        Map<String, Map<String, String>> configurations = kerberosHelper.calculateConfigurations(cluster, null, kerberosDescriptor, true, false);
+
+        PreconfigureServiceType preconfigureServiceType = getPreconfigureServiceType(configurations);
+
+        if (preconfigureServiceType != PreconfigureServiceType.NONE) {
+          Map<String, Map<String, String>> kerberosConfigurations = new HashMap<>();
+          Map<String, Set<String>> propertiesToRemove = new HashMap<>();
+          Map<String, Set<String>> propertiesToIgnore = new HashMap<>();
+
+          if (preconfigureServiceType == PreconfigureServiceType.ALL) {
+            // Force all services to be flagged for pre-configuration...
+            Map<String, KerberosServiceDescriptor> serviceDescriptors = kerberosDescriptor.getServices();
+            if (serviceDescriptors != null) {
+              for (KerberosServiceDescriptor serviceDescriptor : serviceDescriptors.values()) {
+                serviceDescriptor.setPreconfigure(true);
+              }
+            }
+          }
+
+          processServiceComponentHosts(cluster, kerberosDescriptor, configurations, kerberosConfigurations, propertiesToIgnore);
+
+          // Calculate the set of configurations to update and replace any variables
+          // using the previously calculated Map of configurations for the host.
+          kerberosConfigurations = kerberosHelper.processPreconfiguredServiceConfigurations(kerberosConfigurations, configurations, cluster, kerberosDescriptor);
+
+          Map<String, Set<String>> installedServices = calculateInstalledServices(cluster);
+
+          kerberosHelper.applyStackAdvisorUpdates(cluster, installedServices.keySet(), configurations, kerberosConfigurations,
+              propertiesToIgnore, propertiesToRemove, true);
+
+          kerberosHelper.setAuthToLocalRules(cluster, kerberosDescriptor, getDefaultRealm(configurations), installedServices,
+              configurations, kerberosConfigurations, true);
+
+          processConfigurationChanges(cluster, stackId, kerberosDescriptor, kerberosConfigurations, propertiesToRemove, configurations);
+        } else {
+          actionLog.writeStdOut("Skipping: This facility is only available when kerberos-env/preconfigure_services is not \"NONE\"");
+        }
+      } else {
+        actionLog.writeStdOut("Skipping: This facility is only available when Kerberos is enabled");
+      }
+    } else {
+      actionLog.writeStdOut("Skipping: This facility is only available during an upgrade");
+    }
+
+    return createCommandReport(0, HostRoleStatus.COMPLETED, "{}", actionLog.getStdOut(), actionLog.getStdErr());
+  }
+
+  /**
+   * Given a Cluster object creates a map of service names to sets of the installed components for that
+   * service.
+   *
+   * @param cluster the cluster
+   * @return a map of (installed) service names to the relevant set of (installed) component names
+   */
+  private Map<String, Set<String>> calculateInstalledServices(Cluster cluster) {
+    Map<String, Set<String>> installedServices = new HashMap<>();
+    Map<String, Service> services = cluster.getServices();
+
+    for (Service service : services.values()) {
+      installedServices.put(service.getName(), service.getServiceComponents().keySet());
+    }
+
+    return installedServices;
+  }
+
+  /**
+   * Safely retrieves the specified property from the specified configuration type from a map of
+   * configurations.
+   *
+   * @param configurations the existing configurations for the cluster
+   * @return the requested value or null if the configuration does not exist
+   */
+  private String getValueFromConfiguration(Map<String, Map<String, String>> configurations, String configType, String propertyName) {
+    String value = null;
+
+    if (configurations != null) {
+      Map<String, String> kerberosEnv = configurations.get(configType);
+
+      if (kerberosEnv != null) {
+        value = kerberosEnv.get(propertyName);
+      }
+    }
+
+    return value;
+  }
+
+  /**
+   * Safely retrieves the <code>realm</code> property of the <code>kerberos-env</code> configuration.
+   *
+   * @param configurations the existing configurations for the cluster
+   * @return the requested value or null if the configuration does not exist
+   * @see #getValueFromConfiguration(Map, String, String)
+   */
+  private String getDefaultRealm(Map<String, Map<String, String>> configurations) {
+    return getValueFromConfiguration(configurations, KERBEROS_ENV, DEFAULT_REALM);
+  }
+
+  /**
+   * Safely retrieves the <code>preconfigure_services</code> property of the <code>kerberos-env</code> configuration.
+   *
+   * @param configurations the existing configurations for the cluster
+   * @return the requested value or null if the configuration does not exist
+   * @see #getValueFromConfiguration(Map, String, String)
+   */
+  private PreconfigureServiceType getPreconfigureServiceType(Map<String, Map<String, String>> configurations) {
+    String preconfigureServices = getValueFromConfiguration(configurations, KERBEROS_ENV, PRECONFIGURE_SERVICES);
+
+    PreconfigureServiceType preconfigureServiceType = null;
+    if (!StringUtils.isEmpty(preconfigureServices)) {
+      try {
+        preconfigureServiceType = PreconfigureServiceType.valueOf(preconfigureServices.toUpperCase());
+      } catch (Throwable t) {
+        preconfigureServiceType = PreconfigureServiceType.DEFAULT;
+      }
+    }
+
+    return (preconfigureServiceType == null) ? PreconfigureServiceType.DEFAULT : preconfigureServiceType;
+  }
+
+  /**
+   * Determines if upgrade direction is {@link Direction#UPGRADE} or {@link Direction#DOWNGRADE}.
+   *
+   * @return {@code true} if {@link Direction#DOWNGRADE}; {@code false} if {@link Direction#UPGRADE}
+   */
+  private boolean isDowngrade() {
+    return Direction.DOWNGRADE.name().equalsIgnoreCase(getCommandParameterValue(UPGRADE_DIRECTION_KEY));
+  }
+
+  /**
+   * Retrieves the target stack ID for the stack upgrade or downgrade operation.
+   *
+   * @param cluster the cluster
+   * @return the target {@link StackId}
+   * @throws AmbariException if multiple stack id's are detected
+   */
+  private StackId getTargetStackId(Cluster cluster) throws AmbariException {
+    UpgradeContext upgradeContext = getUpgradeContext(cluster);
+
+    // !!! FIXME in a per-service view, what does this become?
+    Set<StackId> stackIds = new HashSet<>();
+
+    for (Service service : cluster.getServices().values()) {
+      RepositoryVersionEntity targetRepoVersion = upgradeContext.getTargetRepositoryVersion(service.getName());
+      StackId targetStackId = targetRepoVersion.getStackId();
+      stackIds.add(targetStackId);
+    }
+
+    if (1 != stackIds.size()) {
+      throw new AmbariException("Services are deployed from multiple stacks and cannot determine a unique one.");
+    }
+
+    return stackIds.iterator().next();
+  }
+
+  /**
+   * Find and iterate through the {@link ServiceComponentHost} objects for the current {@link Cluster}
+   * to calculate property updates and auth-to-local rules.
+   *
+   * @param cluster                the cluster
+   * @param kerberosDescriptor     the Kerberos descriptor
+   * @param currentConfigurations  the current configurations for the cluster
+   * @param kerberosConfigurations the (Kerberos-specific) configuration updates
+   * @param propertiesToBeIgnored  a map to store properties that should be ignored by operations that update property values
+   * @throws AmbariException if an issue occurs
+   */
+  private void processServiceComponentHosts(Cluster cluster, KerberosDescriptor kerberosDescriptor,
+                                            Map<String, Map<String, String>> currentConfigurations,
+                                            Map<String, Map<String, String>> kerberosConfigurations,
+                                            Map<String, Set<String>> propertiesToBeIgnored)
+      throws AmbariException {
+
+    Collection<Host> hosts = cluster.getHosts();
+    if (!hosts.isEmpty()) {
+      // Create the context to use for filtering Kerberos Identities based on the state of the cluster
+      Map<String, Object> filterContext = new HashMap<>();
+      filterContext.put("configurations", currentConfigurations);
+      filterContext.put("services", cluster.getServices().keySet());
+
+      try {
+        Map<String, Set<String>> propertiesToIgnore = null;
+
+        for (Host host : hosts) {
+          // 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 : cluster.getServiceComponentHosts(host.getHostName())) {
+            String hostName = sch.getHostName();
+
+            String serviceName = sch.getServiceName();
+            String componentName = sch.getServiceComponentName();
+
+            KerberosServiceDescriptor serviceDescriptor = kerberosDescriptor.getService(serviceName);
+
+            if (serviceDescriptor != null) {
+              List<KerberosIdentityDescriptor> serviceIdentities = serviceDescriptor.getIdentities(true, filterContext);
+
+              // Add service-level principals (and keytabs)
+              kerberosHelper.addIdentities(null, serviceIdentities,
+                  null, hostName, serviceName, componentName, kerberosConfigurations, currentConfigurations);
+              propertiesToIgnore = gatherPropertiesToIgnore(serviceIdentities, propertiesToIgnore);
+
+              KerberosComponentDescriptor componentDescriptor = serviceDescriptor.getComponent(componentName);
+
+              if (componentDescriptor != null) {
+                List<KerberosIdentityDescriptor> componentIdentities = componentDescriptor.getIdentities(true, filterContext);
+
+                // Calculate the set of configurations to update and replace any variables
+                // using the previously calculated Map of configurations for the host.
+                kerberosHelper.mergeConfigurations(kerberosConfigurations,
+                    componentDescriptor.getConfigurations(true), currentConfigurations, null);
+
+                // Add component-level principals (and keytabs)
+                kerberosHelper.addIdentities(null, componentIdentities,
+                    null, hostName, serviceName, componentName, kerberosConfigurations, currentConfigurations);
+                propertiesToIgnore = gatherPropertiesToIgnore(componentIdentities, propertiesToIgnore);
+              }
+            }
+          }
+        }
+
+        // Add ambari-server identities only if 'kerberos-env.create_ambari_principal = true'
+        if (kerberosHelper.createAmbariIdentities(currentConfigurations.get(KERBEROS_ENV))) {
+          List<KerberosIdentityDescriptor> ambariIdentities = kerberosHelper.getAmbariServerIdentities(kerberosDescriptor);
+
+          for (KerberosIdentityDescriptor identity : ambariIdentities) {
+            // If the identity represents the ambari-server user, use the component name "AMBARI_SERVER_SELF"
+            // so it can be distinguished between other identities related to the AMBARI-SERVER
+            // component.
+            String componentName = KerberosHelper.AMBARI_SERVER_KERBEROS_IDENTITY_NAME.equals(identity.getName())
+                ? "AMBARI_SERVER_SELF"
+                : "AMBARI_SERVER";
+
+            List<KerberosIdentityDescriptor> componentIdentities = Collections.singletonList(identity);
+            kerberosHelper.addIdentities(null, componentIdentities,
+                null, KerberosHelper.AMBARI_SERVER_HOST_NAME, "AMBARI", componentName, kerberosConfigurations, currentConfigurations);
+            propertiesToIgnore = gatherPropertiesToIgnore(componentIdentities, propertiesToIgnore);
+          }
+        }
+
+        if ((propertiesToBeIgnored != null) && (propertiesToIgnore != null)) {
+          propertiesToBeIgnored.putAll(propertiesToIgnore);
+        }
+      } catch (IOException e) {
+        throw new AmbariException(e.getMessage(), e);
+      }
+    }
+  }
+
+  private Map<String, Set<String>> gatherPropertiesToIgnore(List<KerberosIdentityDescriptor> identities,
+                                                            Map<String, Set<String>> propertiesToIgnore) {
+    Map<String, Map<String, String>> identityConfigurations = kerberosHelper.getIdentityConfigurations(identities);
+    if (!MapUtils.isEmpty(identityConfigurations)) {
+      if (propertiesToIgnore == null) {
+        propertiesToIgnore = new HashMap<>();
+      }
+
+      for (Map.Entry<String, Map<String, String>> entry : identityConfigurations.entrySet()) {
+        String configType = entry.getKey();
+        Map<String, String> properties = entry.getValue();
+
+        if (MapUtils.isEmpty(properties)) {
+          Set<String> propertyNames = propertiesToIgnore.get(configType);
+          if (propertyNames == null) {
+            propertyNames = new HashSet<>();
+            propertiesToIgnore.put(configType, propertyNames);
+          }
+          propertyNames.addAll(properties.keySet());
+        }
+      }
+    }
+
+    return propertiesToIgnore;
+  }
+
+  /**
+   * Processes configuration changes to determine if any work needs to be done.
+   * <p/>
+   * If work is to be done, a data file containing the details is created so it they changes may be
+   * processed in the appropriate stage.
+   *
+   * @param cluster                the cluster
+   * @param targetStackId          the target stack id
+   * @param kerberosConfigurations the Kerberos-specific configuration map
+   * @param propertiesToBeRemoved  a map of properties to be removed from the current configuration,
+   *                               grouped by configuration type.
+   * @param variableReplaments     replacement values to use when attempting to perform variable replacements on the property names
+   * @throws AmbariException if an issue is encountered
+   */
+  private void processConfigurationChanges(Cluster cluster, StackId targetStackId,
+                                           KerberosDescriptor kerberosDescriptor,
+                                           Map<String, Map<String, String>> kerberosConfigurations,
+                                           Map<String, Set<String>> propertiesToBeRemoved,
+                                           Map<String, Map<String, String>> variableReplaments)
+      throws AmbariException {
+    actionLog.writeStdOut("Determining configuration changes");
+
+    if (!kerberosConfigurations.isEmpty()) {
+      Map<String, Service> installedServices = cluster.getServices();
+
+      // Build a map of configuration types to properties that indicate which properties should be altered
+      // This map should contain only properties defined in service-level Kerberos descriptors that
+      // have been flagged to be preconfigured and that have not yet been installed.
+      Map<String, Set<String>> propertyFilter = new HashMap<>();
+      Map<String, KerberosServiceDescriptor> serviceDescriptors = kerberosDescriptor.getServices();
+      if (serviceDescriptors != null) {
+        for (KerberosServiceDescriptor serviceDescriptor : serviceDescriptors.values()) {
+          if (!installedServices.containsKey(serviceDescriptor.getName()) && serviceDescriptor.shouldPreconfigure()) {
+            buildFilter(Collections.singleton(serviceDescriptor), propertyFilter, variableReplaments);
+          }
+        }
+      }
+
+      // Add the auth-to-local rule configuration specifications to the filter
+      Map<String, Set<String>> authToLocalProperties = kerberosHelper.translateConfigurationSpecifications(kerberosDescriptor.getAllAuthToLocalProperties());
+      if (!MapUtils.isEmpty(authToLocalProperties)) {
+        for (Map.Entry<String, Set<String>> entry : authToLocalProperties.entrySet()) {
+          Set<String> properties = entry.getValue();
+
+          if (!CollectionUtils.isEmpty(properties)) {
+            String configurationType = entry.getKey();
+
+            Set<String> propertyNames = propertyFilter.get(configurationType);
+            if (propertyNames == null) {
+              propertyNames = new HashSet<>();
+              propertyFilter.put(configurationType, propertyNames);
+            }
+
+            propertyNames.addAll(properties);
+          }
+        }
+      }
+
+      Set<String> visitedTypes = new HashSet<>();
+
+      for (Map.Entry<String, Map<String, String>> entry : kerberosConfigurations.entrySet()) {
+        String configType = entry.getKey();
+
+        String service = cluster.getServiceByConfigType(configType);
+        Set<String> allowedProperties = propertyFilter.get(configType);
+
+        // Update properties for services that are installed and not filtered out
+        if (installedServices.containsKey(service) && !CollectionUtils.isEmpty(allowedProperties)) {
+          Map<String, String> propertiesToUpdate = entry.getValue();
+          Set<String> propertiesToRemove = (propertiesToBeRemoved == null) ? null : propertiesToBeRemoved.get(configType);
+
+          // Filter the properties to update
+          if (propertiesToUpdate != null) {
+            Iterator<Map.Entry<String, String>> mapIterator = propertiesToUpdate.entrySet().iterator();
+            while (mapIterator.hasNext()) {
+              Map.Entry<String, String> mapEntry = mapIterator.next();
+
+              if (!allowedProperties.contains(mapEntry.getKey())) {
+                mapIterator.remove();
+              }
+            }
+          }
+
+          // Filter the properties to remove
+          if (propertiesToRemove != null) {
+            Iterator<String> setIterator = propertiesToRemove.iterator();
+            while (setIterator.hasNext()) {
+              String setEntry = setIterator.next();
+              if (!allowedProperties.contains(setEntry)) {
+                setIterator.remove();
+              }
+            }
+          }
+
+          visitedTypes.add(configType);
+
+          if (!MapUtils.isEmpty(propertiesToUpdate) || !CollectionUtils.isEmpty(propertiesToRemove)) {
+            if (!MapUtils.isEmpty(propertiesToUpdate)) {
+              for (Map.Entry<String, String> property : propertiesToUpdate.entrySet()) {
+                actionLog.writeStdOut(String.format("Setting: %s/%s = %s", configType, property.getKey(), property.getValue()));
+              }
+            }
+
+            if (!CollectionUtils.isEmpty(propertiesToRemove)) {
+              for (String property : propertiesToRemove) {
+                actionLog.writeStdOut(String.format("Removing: %s/%s", configType, property));
+              }
+            }
+
+            configHelper.updateConfigType(cluster, targetStackId,
+                ambariManagementController, configType, propertiesToUpdate, propertiesToRemove,
+                ambariManagementController.getAuthName(), "Preconfiguring for Kerberos during upgrade");
+          }
+        }
+      }
+
+      if (!MapUtils.isEmpty(propertiesToBeRemoved)) {
+        for (Map.Entry<String, Set<String>> entry : propertiesToBeRemoved.entrySet()) {
+          String configType = entry.getKey();
+
+          if (!visitedTypes.contains(configType)) {
+            Set<String> propertiesToRemove = entry.getValue();
+
+            if (!CollectionUtils.isEmpty(propertiesToRemove)) {
+              for (String property : propertiesToRemove) {
+                actionLog.writeStdOut(String.format("Removing: %s/%s", configType, property));
+              }
+
+              configHelper.updateConfigType(cluster, targetStackId,
+                  ambariManagementController, configType, null, entry.getValue(),
+                  ambariManagementController.getAuthName(), "Preconfiguring for Kerberos during upgrade");
+            }
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Adds entries to the property filter (<code>propertyFilter</code>) found in the {@link KerberosConfigurationDescriptor}s
+   * within the specified node of the Kerberos descriptor.
+   *
+   * @param containers     the Kerberos descriptor containers to process
+   * @param propertyFilter the property filter map to update
+   * @param replacements   replacement values to use when attempting to perform variable replacements on the property names
+   * @throws AmbariException if an issue occurs while replacing variables in the property names
+   */
+  private void buildFilter(Collection<? extends AbstractKerberosDescriptorContainer> containers,
+                           Map<String, Set<String>> propertyFilter,
+                           Map<String, Map<String, String>> replacements)
+      throws AmbariException {
+    if (containers != null) {
+      for (AbstractKerberosDescriptorContainer container : containers) {
+        Map<String, KerberosConfigurationDescriptor> configurationDescriptors = container.getConfigurations(false);
+
+        if (!MapUtils.isEmpty(configurationDescriptors)) {
+          for (KerberosConfigurationDescriptor configurationDescriptor : configurationDescriptors.values()) {
+            Map<String, String> properties = configurationDescriptor.getProperties();
+
+            if (!MapUtils.isEmpty(properties)) {
+              String configType = configurationDescriptor.getType();
+
+              Set<String> propertyNames = propertyFilter.get(configType);
+              if (propertyNames == null) {
+                propertyNames = new HashSet<>();
+                propertyFilter.put(configType, propertyNames);
+              }
+
+              // Replace variables in the property name. For example ${knox-env/knox_user}.
+              for (String propertyName : properties.keySet()) {
+                propertyNames.add(variableReplacementHelper.replaceVariables(propertyName, replacements));
+              }
+            }
+          }
+        }
+
+        Collection<? extends AbstractKerberosDescriptorContainer> childContainers = container.getChildContainers();
+        if (childContainers != null) {
+          buildFilter(childContainers, propertyFilter, replacements);
+        }
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/ambari/blob/5230d935/ambari-server/src/main/resources/stacks/HDP/2.5/upgrades/nonrolling-upgrade-2.6.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.5/upgrades/nonrolling-upgrade-2.6.xml b/ambari-server/src/main/resources/stacks/HDP/2.5/upgrades/nonrolling-upgrade-2.6.xml
index 492c308..90a7d97 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.5/upgrades/nonrolling-upgrade-2.6.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.5/upgrades/nonrolling-upgrade-2.6.xml
@@ -669,6 +669,18 @@
     </group>
 
     <!--
+    After processing this group, items declared to be preconfigured will be applied to the existing
+    configurations.
+    -->
+    <group xsi:type="cluster" name="PRECONFIGURE_COMMON_KERBEROS_PROPERTIES" title="Preconfigure Kerberos-related properties">
+      <condition xsi:type="security" type="kerberos"/>
+      <direction>UPGRADE</direction>
+      <execute-stage title="Preconfigure Kerberos-related properties">
+        <task xsi:type="server_action" class="org.apache.ambari.server.serveraction.upgrades.PreconfigureKerberosAction"/>
+      </execute-stage>
+    </group>
+
+    <!--
     Invoke "hdp-select set all" to change any components we may have missed
     that are installed on the hosts but not known by Ambari.
     -->

http://git-wip-us.apache.org/repos/asf/ambari/blob/5230d935/ambari-server/src/main/resources/stacks/HDP/2.5/upgrades/upgrade-2.6.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.5/upgrades/upgrade-2.6.xml b/ambari-server/src/main/resources/stacks/HDP/2.5/upgrades/upgrade-2.6.xml
index 995a1d3..5ee82c3 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.5/upgrades/upgrade-2.6.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.5/upgrades/upgrade-2.6.xml
@@ -148,6 +148,18 @@
       </execute-stage>
     </group>
 
+    <!--
+    After processing this group, items declared to be preconfigured will be applied to the existing
+    configurations.
+    -->
+    <group xsi:type="cluster" name="PRECONFIGURE_COMMON_KERBEROS_PROPERTIES" title="Preconfigure Kerberos-related properties">
+      <condition xsi:type="security" type="kerberos"/>
+      <direction>UPGRADE</direction>
+      <execute-stage title="Preconfigure Kerberos-related properties">
+        <task xsi:type="server_action" class="org.apache.ambari.server.serveraction.upgrades.PreconfigureKerberosAction"/>
+      </execute-stage>
+    </group>
+
     <group name="CORE_MASTER" title="Core Masters">
       <service-check>false</service-check>
       <service name="HDFS">