You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by dm...@apache.org on 2015/09/09 17:32:50 UTC

[4/4] ambari git commit: AMBARI-12700. Stop-and-Start Upgrade: Move Configs out of Upgrade Pack (dlysnichenko)

AMBARI-12700. Stop-and-Start Upgrade: Move Configs out of Upgrade Pack (dlysnichenko)


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

Branch: refs/heads/branch-dev-stop-all-upgrade
Commit: a67ddd27d2f8e164d55df02b813904e64753772a
Parents: 54146bb
Author: Lisnichenko Dmitro <dl...@hortonworks.com>
Authored: Wed Sep 9 18:32:40 2015 +0300
Committer: Lisnichenko Dmitro <dl...@hortonworks.com>
Committed: Wed Sep 9 18:32:40 2015 +0300

----------------------------------------------------------------------
 .../server/api/services/AmbariMetaInfo.java     |  21 +-
 .../internal/UpgradeResourceProvider.java       |  47 +-
 .../serveraction/upgrades/ConfigureAction.java  |  23 +-
 .../server/stack/ModuleFileUnmarshaller.java    |   4 +-
 .../server/stack/StackDefinitionDirectory.java  |   2 +
 .../ambari/server/stack/StackDirectory.java     |  53 +-
 .../apache/ambari/server/stack/StackModule.java |   2 +-
 .../apache/ambari/server/state/StackInfo.java   |  29 +-
 .../ambari/server/state/UpgradeHelper.java      |  15 +-
 .../server/state/stack/ConfigUpgradePack.java   | 147 ++++
 .../upgrade/ConfigUpgradeChangeDefinition.java  | 420 ++++++++++
 .../state/stack/upgrade/ConfigureTask.java      | 328 ++------
 .../server/state/stack/upgrade/Grouping.java    |   2 +-
 .../stack/upgrade/StageWrapperBuilder.java      |   2 +-
 .../stacks/HDP/2.2/upgrades/config-upgrade.xml  |  55 ++
 .../stacks/HDP/2.2/upgrades/upgrade-2.2.xml     |  26 +-
 .../stacks/HDP/2.2/upgrades/upgrade-2.3.xml     | 621 ++------------
 .../stacks/HDP/2.3/upgrades/config-upgrade.xml  | 805 +++++++++++++++++++
 .../stacks/HDP/2.3/upgrades/upgrade-2.3.xml     |  26 +-
 .../upgrades/ConfigureActionTest.java           |  36 +-
 .../ambari/server/state/UpgradeHelperTest.java  | 395 ++++-----
 .../server/state/stack/UpgradePackTest.java     | 212 ++---
 22 files changed, 2005 insertions(+), 1266 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java b/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
index 4afa9b0..1e74dfb 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
@@ -60,6 +60,7 @@ import org.apache.ambari.server.state.kerberos.KerberosServiceDescriptorFactory;
 import org.apache.ambari.server.state.stack.Metric;
 import org.apache.ambari.server.state.stack.MetricDefinition;
 import org.apache.ambari.server.state.stack.OsFamily;
+import org.apache.ambari.server.state.stack.ConfigUpgradePack;
 import org.apache.ambari.server.state.stack.UpgradePack;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -994,7 +995,7 @@ public class AmbariMetaInfo {
     }
 
     return alertDefinitionFactory.getAlertDefinitions(alertsFile,
-        service.getName());
+            service.getName());
   }
 
   /**
@@ -1203,6 +1204,24 @@ public class AmbariMetaInfo {
   }
 
   /**
+   * Get all upgrade config pack if it is available for a stack.
+   *
+   * @param stackName the stack name
+   * @param stackVersion the stack version
+   * @return config upgrade pack for stack or null if it is
+   * not defined for stack
+   */
+  public ConfigUpgradePack getConfigUpgradePack(String stackName, String stackVersion) {
+    try {
+      StackInfo stack = getStack(stackName, stackVersion);
+      return stack.getConfigUpgradePack();
+    } catch (AmbariException e) {
+      LOG.debug("Cannot load config upgrade pack for non-existent stack {}-{}", stackName, stackVersion, e);
+      return null;
+    }
+  }
+
+  /**
    * Gets the fully compiled Kerberos descriptor for the relevant stack and version.
    * <p/>
    * All of the kerberos.json files from the specified stack (and version) are read, parsed and

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
index dddec73..2c9714e 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
@@ -85,6 +85,7 @@ import org.apache.ambari.server.state.StackInfo;
 import org.apache.ambari.server.state.UpgradeContext;
 import org.apache.ambari.server.state.UpgradeHelper;
 import org.apache.ambari.server.state.UpgradeHelper.UpgradeGroupHolder;
+import org.apache.ambari.server.state.stack.ConfigUpgradePack;
 import org.apache.ambari.server.state.stack.UpgradePack;
 import org.apache.ambari.server.state.stack.upgrade.ConfigureTask;
 import org.apache.ambari.server.state.stack.upgrade.Direction;
@@ -488,10 +489,10 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
     }
 
     Map<String, UpgradePack> packs = s_metaProvider.get().getUpgradePacks(stack.getStackName(),
-        stack.getStackVersion());
+            stack.getStackVersion());
 
     UpgradePack pack = null;
-    if (!preferredUpgradePackName.isEmpty() && packs.containsKey(preferredUpgradePackName)) {
+    if (preferredUpgradePackName != null && !preferredUpgradePackName.isEmpty() && packs.containsKey(preferredUpgradePackName)) {
       pack = packs.get(preferredUpgradePackName);
     }
 
@@ -617,6 +618,11 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
     // names are read and set on the command for filling in later
     processConfigurations(targetStackId.getStackName(), cluster, version, direction, pack);
 
+    // TODO: for cross-stack upgrade, merge a new config upgrade pack from all
+    // target stacks involved into upgrade and pass it into method
+    ConfigUpgradePack configUpgradePack = s_metaProvider.get().getConfigUpgradePack(
+            targetStackId.getStackName(), targetStackId.getStackVersion());
+
     for (UpgradeGroupHolder group : groups) {
       UpgradeGroupEntity groupEntity = new UpgradeGroupEntity();
       groupEntity.setName(group.name);
@@ -642,7 +648,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
               injectVariables(configHelper, cluster, itemEntity);
 
               makeServerSideStage(ctx, req, itemEntity, (ServerSideActionTask) task, skippable,
-                  allowRetry);
+                  allowRetry, configUpgradePack);
             }
           }
         } else {
@@ -667,7 +673,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
     entity.setFromVersion(cluster.getCurrentClusterVersion().getRepositoryVersion().getVersion());
     entity.setToVersion(version);
     entity.setUpgradeGroups(groupEntities);
-    entity.setClusterId(Long.valueOf(cluster.getClusterId()));
+    entity.setClusterId(cluster.getClusterId());
     entity.setDirection(direction);
     entity.setUpgradePackage(pack.getName());
     entity.setUpgradeType(pack.getType());
@@ -1132,8 +1138,22 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
     request.addStages(Collections.singletonList(stage));
   }
 
+  /**
+   * Creates a stage consisting of server side actions
+   * @param context upgrade context
+   * @param request upgrade request
+   * @param entity a single of upgrade
+   * @param task server-side task (if any)
+   * @param skippable if user can skip stage on failure
+   * @param allowRetry if user can retry running stage on failure
+   * @param configUpgradePack a runtime-generated config upgrade pack that
+   * contains all config change definitions from all stacks involved into
+   * upgrade
+   * @throws AmbariException
+   */
   private void makeServerSideStage(UpgradeContext context, RequestStageContainer request,
-      UpgradeItemEntity entity, ServerSideActionTask task, boolean skippable, boolean allowRetry)
+      UpgradeItemEntity entity, ServerSideActionTask task, boolean skippable, boolean allowRetry,
+      ConfigUpgradePack configUpgradePack)
           throws AmbariException {
 
     Cluster cluster = context.getCluster();
@@ -1166,7 +1186,8 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
       }
       case CONFIGURE: {
         ConfigureTask ct = (ConfigureTask) task;
-        Map<String, String> configurationChanges = ct.getConfigurationChanges(cluster);
+        Map<String, String> configurationChanges =
+                ct.getConfigurationChanges(cluster, configUpgradePack);
 
         // add all configuration changes to the command params
         commandParams.putAll(configurationChanges);
@@ -1219,13 +1240,13 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
     entity.setStageId(Long.valueOf(stageId));
 
     stage.addServerActionCommand(task.getImplementationClass(),
-        getManagementController().getAuthName(),
-        Role.AMBARI_SERVER_ACTION,
-        RoleCommand.EXECUTE,
-        cluster.getClusterName(),
-        new ServiceComponentHostServerActionEvent(null,
-            System.currentTimeMillis()),
-        commandParams, itemDetail, null, Integer.valueOf(1200), allowRetry);
+            getManagementController().getAuthName(),
+            Role.AMBARI_SERVER_ACTION,
+            RoleCommand.EXECUTE,
+            cluster.getClusterName(),
+            new ServiceComponentHostServerActionEvent(null,
+                    System.currentTimeMillis()),
+            commandParams, itemDetail, null, Integer.valueOf(1200), allowRetry);
 
     request.addStages(Collections.singletonList(stage));
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ConfigureAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ConfigureAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ConfigureAction.java
index c717582..ef21a2a 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ConfigureAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ConfigureAction.java
@@ -46,7 +46,10 @@ import org.apache.ambari.server.state.DesiredConfig;
 import org.apache.ambari.server.state.PropertyInfo;
 import org.apache.ambari.server.state.StackId;
 import org.apache.ambari.server.state.stack.upgrade.ConfigureTask;
-import org.apache.ambari.server.state.stack.upgrade.ConfigureTask.ConfigurationKeyValue;
+import org.apache.ambari.server.state.stack.upgrade.ConfigUpgradeChangeDefinition.ConfigurationKeyValue;
+import org.apache.ambari.server.state.stack.upgrade.ConfigUpgradeChangeDefinition.Transfer;
+import org.apache.ambari.server.state.stack.upgrade.ConfigUpgradeChangeDefinition.Replace;
+import org.apache.ambari.server.state.stack.upgrade.ConfigUpgradeChangeDefinition.Masked;
 import org.apache.commons.lang.StringUtils;
 
 import com.google.gson.Gson;
@@ -176,27 +179,27 @@ public class ConfigureAction extends AbstractServerAction {
     String configType = commandParameters.get(ConfigureTask.PARAMETER_CONFIG_TYPE);
 
     // extract transfers
-    List<ConfigureTask.ConfigurationKeyValue> keyValuePairs = Collections.emptyList();
+    List<ConfigurationKeyValue> keyValuePairs = Collections.emptyList();
     String keyValuePairJson = commandParameters.get(ConfigureTask.PARAMETER_KEY_VALUE_PAIRS);
     if (null != keyValuePairJson) {
       keyValuePairs = m_gson.fromJson(
-          keyValuePairJson, new TypeToken<List<ConfigureTask.ConfigurationKeyValue>>(){}.getType());
+          keyValuePairJson, new TypeToken<List<ConfigurationKeyValue>>(){}.getType());
     }
 
     // extract transfers
-    List<ConfigureTask.Transfer> transfers = Collections.emptyList();
+    List<Transfer> transfers = Collections.emptyList();
     String transferJson = commandParameters.get(ConfigureTask.PARAMETER_TRANSFERS);
     if (null != transferJson) {
       transfers = m_gson.fromJson(
-        transferJson, new TypeToken<List<ConfigureTask.Transfer>>(){}.getType());
+        transferJson, new TypeToken<List<Transfer>>(){}.getType());
     }
 
     // extract replacements
-    List<ConfigureTask.Replace> replacements = Collections.emptyList();
+    List<Replace> replacements = Collections.emptyList();
     String replaceJson = commandParameters.get(ConfigureTask.PARAMETER_REPLACEMENTS);
     if (null != replaceJson) {
       replacements = m_gson.fromJson(
-          replaceJson, new TypeToken<List<ConfigureTask.Replace>>(){}.getType());
+          replaceJson, new TypeToken<List<Replace>>(){}.getType());
     }
 
     // if there is nothing to do, then skip the task
@@ -240,7 +243,7 @@ public class ConfigureAction extends AbstractServerAction {
 
     // !!! do transfers first before setting defined values
     StringBuilder outputBuffer = new StringBuilder(250);
-    for (ConfigureTask.Transfer transfer : transfers) {
+    for (Transfer transfer : transfers) {
       switch (transfer.operation) {
         case COPY:
           String valueToCopy = null;
@@ -400,7 +403,7 @@ public class ConfigureAction extends AbstractServerAction {
     }
 
     // !!! string replacements happen only on the new values.
-    for (ConfigureTask.Replace replacement : replacements) {
+    for (Replace replacement : replacements) {
       if (newValues.containsKey(replacement.key)) {
         String toReplace = newValues.get(replacement.key);
 
@@ -534,7 +537,7 @@ public class ConfigureAction extends AbstractServerAction {
     return result;
   }
 
-  private static String mask(ConfigureTask.Masked mask, String value) {
+  private static String mask(Masked mask, String value) {
     if (mask.mask) {
       return StringUtils.repeat("*", value.length());
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/stack/ModuleFileUnmarshaller.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/stack/ModuleFileUnmarshaller.java b/ambari-server/src/main/java/org/apache/ambari/server/stack/ModuleFileUnmarshaller.java
index aa8e17b..9e2f997 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/stack/ModuleFileUnmarshaller.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/stack/ModuleFileUnmarshaller.java
@@ -18,6 +18,7 @@
 
 package org.apache.ambari.server.stack;
 
+import org.apache.ambari.server.state.stack.ConfigUpgradePack;
 import org.apache.ambari.server.state.stack.ConfigurationXml;
 import org.apache.ambari.server.state.stack.RepositoryXml;
 import org.apache.ambari.server.state.stack.ServiceMetainfoXml;
@@ -63,12 +64,13 @@ class ModuleFileUnmarshaller {
     try {
       // three classes define the top-level element "metainfo", so we need 3 contexts.
       JAXBContext ctx = JAXBContext.newInstance(StackMetainfoXml.class, RepositoryXml.class,
-          ConfigurationXml.class, UpgradePack.class);
+          ConfigurationXml.class, UpgradePack.class, ConfigUpgradePack.class);
 
       jaxbContexts.put(StackMetainfoXml.class, ctx);
       jaxbContexts.put(RepositoryXml.class, ctx);
       jaxbContexts.put(ConfigurationXml.class, ctx);
       jaxbContexts.put(UpgradePack.class, ctx);
+      jaxbContexts.put(ConfigUpgradePack.class, ctx);
       jaxbContexts.put(ServiceMetainfoXml.class, JAXBContext.newInstance(ServiceMetainfoXml.class));
     } catch (JAXBException e) {
       throw new RuntimeException (e);

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/stack/StackDefinitionDirectory.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/stack/StackDefinitionDirectory.java b/ambari-server/src/main/java/org/apache/ambari/server/stack/StackDefinitionDirectory.java
index 8f81b5a..c739211 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/stack/StackDefinitionDirectory.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/stack/StackDefinitionDirectory.java
@@ -37,6 +37,8 @@ public abstract class StackDefinitionDirectory {
     }
   };
 
+  protected static final String CONFIG_UPGRADE_XML_FILENAME_PREFIX = "config-upgrade.xml";
+
   /**
    * underlying directory
    */

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/stack/StackDirectory.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/stack/StackDirectory.java b/ambari-server/src/main/java/org/apache/ambari/server/stack/StackDirectory.java
index db947ca..515d031 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/stack/StackDirectory.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/stack/StackDirectory.java
@@ -23,6 +23,7 @@ import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.state.stack.RepositoryXml;
 import org.apache.ambari.server.state.stack.StackMetainfoXml;
 import org.apache.ambari.server.state.stack.StackRoleCommandOrder;
+import org.apache.ambari.server.state.stack.ConfigUpgradePack;
 import org.apache.ambari.server.state.stack.UpgradePack;
 import org.apache.commons.io.FilenameUtils;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -98,6 +99,11 @@ public class StackDirectory extends StackDefinitionDirectory {
   private Map<String, UpgradePack> upgradePacks;
 
   /**
+   * Config delta from prev stack
+   */
+  private ConfigUpgradePack configUpgradePack;
+
+  /**
    * metainfo file representation
    */
   private StackMetainfoXml metaInfoXml;
@@ -254,6 +260,13 @@ public class StackDirectory extends StackDefinitionDirectory {
   }
 
   /**
+   * @return Config delta from prev stack or null if no config upgrade patches available
+   */
+  public ConfigUpgradePack getConfigUpgradePack() {
+    return configUpgradePack;
+  }
+
+  /**
    * Obtain the object representation of the stack role_command_order.json file
    *
    * @return object representation of the stack role_command_order.json file
@@ -404,20 +417,35 @@ public class StackDirectory extends StackDefinitionDirectory {
    * @throws AmbariException if unable to parse stack upgrade file
    */
   private void parseUpgradePacks(Collection<String> subDirs) throws AmbariException {
-    Map<String, UpgradePack> upgradeMap = new HashMap<String, UpgradePack>();
+    Map<String, UpgradePack> upgradeMap = new HashMap<>();
+    ConfigUpgradePack configUpgradePack = null;
     if (subDirs.contains(UPGRADE_PACK_FOLDER_NAME)) {
       File f = new File(getAbsolutePath() + File.separator + UPGRADE_PACK_FOLDER_NAME);
       if (f.isDirectory()) {
         upgradesDir = f.getAbsolutePath();
         for (File upgradeFile : f.listFiles(XML_FILENAME_FILTER)) {
-          try {
-            String upgradePackName = FilenameUtils.removeExtension(upgradeFile.getName());
-            UpgradePack pack = unmarshaller.unmarshal(UpgradePack.class, upgradeFile);
-            pack.setName(upgradePackName);
-            upgradeMap.put(upgradePackName, pack);
-          } catch (JAXBException e) {
-            throw new AmbariException("Unable to parse stack upgrade file at location: " +
-                upgradeFile.getAbsolutePath(), e);
+          if (upgradeFile.getName().toLowerCase().startsWith(CONFIG_UPGRADE_XML_FILENAME_PREFIX)) {
+            try { // Parse config upgrade pack
+              if (configUpgradePack == null) {
+                configUpgradePack = unmarshaller.unmarshal(ConfigUpgradePack.class, upgradeFile);
+              } else { // If user messed things up with lower/upper case filenames
+                throw new AmbariException(String.format("There are multiple files with name like %s" +
+                        upgradeFile.getAbsolutePath()));
+              }
+            } catch (JAXBException e) {
+              throw new AmbariException("Unable to parse stack upgrade file at location: " +
+                      upgradeFile.getAbsolutePath(), e);
+            }
+          } else {
+            try {
+              String upgradePackName = FilenameUtils.removeExtension(upgradeFile.getName());
+              UpgradePack pack = unmarshaller.unmarshal(UpgradePack.class, upgradeFile);
+              pack.setName(upgradePackName);
+              upgradeMap.put(upgradePackName, pack);
+            } catch (JAXBException e) {
+              throw new AmbariException("Unable to parse stack upgrade file at location: " +
+                      upgradeFile.getAbsolutePath(), e);
+            }
           }
         }
       }
@@ -430,6 +458,13 @@ public class StackDirectory extends StackDefinitionDirectory {
     if (! upgradeMap.isEmpty()) {
       upgradePacks = upgradeMap;
     }
+
+    if (configUpgradePack != null) {
+      this.configUpgradePack = configUpgradePack;
+    } else {
+      LOG.info("Stack '{}' doesn't contain config upgrade pack file", getPath());
+    }
+
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/stack/StackModule.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/stack/StackModule.java b/ambari-server/src/main/java/org/apache/ambari/server/stack/StackModule.java
index 4b88aff..1d7da2d 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/stack/StackModule.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/stack/StackModule.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -420,6 +419,7 @@ public class StackModule extends BaseModule<StackModule, StackInfo> implements V
       stackInfo.setWidgetsDescriptorFileLocation(stackDirectory.getWidgetsDescriptorFilePath());
       stackInfo.setUpgradesFolder(stackDirectory.getUpgradesDir());
       stackInfo.setUpgradePacks(stackDirectory.getUpgradePacks());
+      stackInfo.setConfigUpgradePack(stackDirectory.getConfigUpgradePack());
       stackInfo.setRoleCommandOrder(stackDirectory.getRoleCommandOrder());
       populateConfigurationModules();
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/state/StackInfo.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/StackInfo.java b/ambari-server/src/main/java/org/apache/ambari/server/state/StackInfo.java
index 87301e5..2499c4c 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/StackInfo.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/StackInfo.java
@@ -31,6 +31,7 @@ import java.util.Set;
 import org.apache.ambari.server.controller.StackVersionResponse;
 import org.apache.ambari.server.stack.Validable;
 import org.apache.ambari.server.state.stack.StackRoleCommandOrder;
+import org.apache.ambari.server.state.stack.ConfigUpgradePack;
 import org.apache.ambari.server.state.stack.UpgradePack;
 
 public class StackInfo implements Comparable<StackInfo>, Validable{
@@ -48,6 +49,7 @@ public class StackInfo implements Comparable<StackInfo>, Validable{
   private List<PropertyInfo> properties;
   private Map<String, Map<String, Map<String, String>>> configTypes;
   private Map<String, UpgradePack> upgradePacks;
+  private ConfigUpgradePack configUpgradePack;
   private StackRoleCommandOrder roleCommandOrder;
   private boolean valid = true;
 
@@ -354,6 +356,15 @@ public class StackInfo implements Comparable<StackInfo>, Validable{
   }
 
   /**
+   * Obtain all stack upgrade packs.
+   *
+   * @return map of upgrade pack name to upgrade pack or {@code null} if no packs
+   */
+  public Map<String, UpgradePack> getUpgradePacks() {
+    return upgradePacks;
+  }
+
+  /**
    * Set upgrade packs.
    *
    * @param upgradePacks map of upgrade packs
@@ -363,14 +374,22 @@ public class StackInfo implements Comparable<StackInfo>, Validable{
   }
 
   /**
-   * Obtain all stack upgrade packs.
-   *
-   * @return map of upgrade pack name to upgrade pack or {@code null} of no packs
+   * Get config upgrade pack for stack
+   * @return config upgrade pack for stack or null if it is
+   * not defined
    */
-  public Map<String, UpgradePack> getUpgradePacks() {
-    return upgradePacks;
+  public ConfigUpgradePack getConfigUpgradePack() {
+    return configUpgradePack;
   }
 
+  /**
+   * Set config upgrade pack for stack
+   * @param configUpgradePack config upgrade pack for stack or null if it is
+   * not defined
+   */
+  public void setConfigUpgradePack(ConfigUpgradePack configUpgradePack) {
+    this.configUpgradePack = configUpgradePack;
+  }
 
   @Override
   public int compareTo(StackInfo o) {

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/state/UpgradeHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/UpgradeHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/state/UpgradeHelper.java
index ecefe6e..442c9ed 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/UpgradeHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/UpgradeHelper.java
@@ -47,6 +47,7 @@ import org.apache.ambari.server.stack.HostsType;
 import org.apache.ambari.server.stack.MasterHostResolver;
 import org.apache.ambari.server.state.stack.UpgradePack;
 import org.apache.ambari.server.state.stack.UpgradePack.ProcessingComponent;
+import org.apache.ambari.server.state.stack.upgrade.ConfigureTask;
 import org.apache.ambari.server.state.stack.upgrade.Direction;
 import org.apache.ambari.server.state.stack.upgrade.Grouping;
 import org.apache.ambari.server.state.stack.upgrade.ManualTask;
@@ -199,7 +200,7 @@ public class UpgradeHelper {
 
     // Note, only a Rolling Upgrade uses processing tasks.
     Map<String, Map<String, ProcessingComponent>> allTasks = upgradePack.getTasks();
-    List<UpgradeGroupHolder> groups = new ArrayList<UpgradeGroupHolder>();
+    List<UpgradeGroupHolder> groups = new ArrayList<>();
 
     for (Grouping group : upgradePack.getGroups(context.getDirection())) {
 
@@ -227,7 +228,7 @@ public class UpgradeHelper {
       // Rolling Downgrade must reverse the order of services.
       if (upgradePack.getType() == UpgradeType.ROLLING) {
         if (context.getDirection().isDowngrade() && !services.isEmpty()) {
-          List<UpgradePack.OrderService> reverse = new ArrayList<UpgradePack.OrderService>(services);
+          List<UpgradePack.OrderService> reverse = new ArrayList<>(services);
           Collections.reverse(reverse);
           services = reverse;
         }
@@ -235,7 +236,7 @@ public class UpgradeHelper {
 
       // !!! cluster and service checks are empty here
       for (UpgradePack.OrderService service : services) {
-      
+
         if (upgradePack.getType() == UpgradeType.ROLLING && !allTasks.containsKey(service.serviceName)) {
           continue;
         }
@@ -278,7 +279,7 @@ public class UpgradeHelper {
             if (null != functionName) {
               pc = new ProcessingComponent();
               pc.name = component;
-              pc.tasks = new ArrayList<Task>();
+              pc.tasks = new ArrayList<>();
 
               if (functionName == Type.START) {
                 pc.tasks.add(new StartTask());
@@ -302,7 +303,7 @@ public class UpgradeHelper {
             // !!! revisit if needed
             if (!hostsType.hosts.isEmpty() && hostsType.master != null && hostsType.secondary != null) {
               // The order is important, first do the standby, then the active namenode.
-              LinkedHashSet<String> order = new LinkedHashSet<String>();
+              LinkedHashSet<String> order = new LinkedHashSet<>();
 
               order.add(hostsType.secondary);
               order.add(hostsType.master);
@@ -398,7 +399,7 @@ public class UpgradeHelper {
 
     String result = source;
 
-    List<String> tokens = new ArrayList<String>(5);
+    List<String> tokens = new ArrayList<>(5);
     Matcher matcher = PLACEHOLDER_REGEX.matcher(source);
     while (matcher.find()) {
       tokens.add(matcher.group(1));
@@ -494,7 +495,7 @@ public class UpgradeHelper {
     /**
      * List of stages for the group
      */
-    public List<StageWrapper> items = new ArrayList<StageWrapper>();
+    public List<StageWrapper> items = new ArrayList<>();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/state/stack/ConfigUpgradePack.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/ConfigUpgradePack.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/ConfigUpgradePack.java
new file mode 100644
index 0000000..2896255
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/ConfigUpgradePack.java
@@ -0,0 +1,147 @@
+/**
+ * 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.state.stack;
+
+import org.apache.ambari.server.state.stack.upgrade.ConfigUpgradeChangeDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Represents a pack of changes that should be applied to configs
+ * when upgrading from a previous stack. In other words, it's a config delta
+ * from prev stack
+ */
+@XmlRootElement(name="upgrade-config-changes")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ConfigUpgradePack {
+
+  /**
+   * Defines per-service config changes
+   */
+  @XmlElementWrapper(name="services")
+  @XmlElement(name="service")
+  private List<AffectedService> services;
+
+  /**
+   * Contains a cached mapping of <change id, change definition>.
+   */
+  private Map<String, ConfigUpgradeChangeDefinition> changesById;
+
+  private static Logger LOG = LoggerFactory.getLogger(ConfigUpgradePack.class);
+
+  /**
+   * no-arg default constructor for JAXB
+   */
+  public ConfigUpgradePack() {
+  }
+
+  public ConfigUpgradePack(List<AffectedService> services) {
+    this.services = services;
+  }
+
+  /**
+   * @return a list of per-service config changes. List should not be modified
+   * in runtime, since it will make cache stale.
+   */
+  public List<AffectedService> getServices() {
+    return services;
+  }
+
+  /**
+   * @return a map of <service name, AffectedService>.
+   */
+  public Map<String, AffectedService> getServiceMap() {
+    Map<String, AffectedService> result = new HashMap<>();
+    for (AffectedService service : services) {
+      result.put(service.name, service);
+    }
+    return result;
+  }
+
+  /**
+   * @return a map of <change id, change definition>. Map is built once and
+   * cached
+   */
+  public Map<String, ConfigUpgradeChangeDefinition> enumerateConfigChangesByID() {
+    if (changesById == null) {
+      changesById = new HashMap<>();
+      for(AffectedService service : services) {
+        for(AffectedComponent component: service.components) {
+          for (ConfigUpgradeChangeDefinition changeDefinition : component.changes) {
+            if (changeDefinition.id == null) {
+              LOG.warn(String.format("Config upgrade change definition for service %s," +
+                      " component %s has no id", service.name, component.name));
+            } else if (changesById.containsKey(changeDefinition.id)) {
+              LOG.warn("Duplicate config upgrade change definition with ID " +
+                      changeDefinition.id);
+            }
+            changesById.put(changeDefinition.id, changeDefinition);
+          }
+        }
+      }
+    }
+    return changesById;
+  }
+
+  /**
+   * A service definition in the 'services' element.
+   */
+  public static class AffectedService {
+
+    @XmlAttribute
+    public String name;
+
+    @XmlElement(name="component")
+    public List<AffectedComponent> components;
+
+    /**
+     * @return a map of <component name, AffectedService>
+     */
+    public Map<String, AffectedComponent> getComponentMap() {
+      Map<String, AffectedComponent> result = new HashMap<>();
+      for (AffectedComponent component : components) {
+        result.put(component.name, component);
+      }
+      return result;
+    }
+  }
+
+  /**
+   * A component definition in the 'services/service' path.
+   */
+  public static class AffectedComponent {
+
+    @XmlAttribute
+    public String name;
+
+    @XmlElementWrapper(name="changes")
+    @XmlElement(name="definition")
+    public List<ConfigUpgradeChangeDefinition> changes;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConfigUpgradeChangeDefinition.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConfigUpgradeChangeDefinition.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConfigUpgradeChangeDefinition.java
new file mode 100644
index 0000000..780f96d
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConfigUpgradeChangeDefinition.java
@@ -0,0 +1,420 @@
+/**
+ * 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.state.stack.upgrade;
+
+import com.google.gson.Gson;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * The {@link ConfigUpgradeChangeDefinition} represents a configuration change. This change can be
+ * defined with conditional statements that will only set values if a condition
+ * passes:
+ * <p/>
+ *
+ * <pre>
+ * {@code
+ * <definition>
+ *   <condition type="hive-site" key="hive.server2.transport.mode" value="binary">
+ *     <type>hive-site</type>
+ *     <key>hive.server2.thrift.port</key>
+ *     <value>10010</value>
+ *   </condition>
+ *   <condition type="hive-site" key="hive.server2.transport.mode" value="http">
+ *     <type>hive-site</type>
+ *     <key>hive.server2.http.port</key>
+ *     <value>10011</value>
+ *   </condition>
+ * </definition>
+ * }
+ * </pre>
+ *
+ * It's also possible to simple set values directly without a precondition
+ * check.
+ *
+ * <pre>
+ * {@code
+ * <definition xsi:type="configure">
+ *   <type>hive-site</type>
+ *   <set key="hive.server2.thrift.port" value="10010"/>
+ *   <set key="foo" value="bar"/>
+ *   <set key="foobar" value="baz"/>
+ * </definition>
+ * }
+ * </pre>
+ *
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ConfigUpgradeChangeDefinition {
+
+  private static Logger LOG = LoggerFactory.getLogger(ConfigUpgradeChangeDefinition.class);
+
+  /**
+   * The key that represents the configuration type to change (ie hdfs-site).
+   */
+  public static final String PARAMETER_CONFIG_TYPE = "configure-task-config-type";
+
+  /**
+   * Setting key/value pairs can be several per task, so they're passed in as a
+   * json-ified list of objects.
+   */
+  public static final String PARAMETER_KEY_VALUE_PAIRS = "configure-task-key-value-pairs";
+
+  /**
+   * Transfers can be several per task, so they're passed in as a json-ified
+   * list of objects.
+   */
+  public static final String PARAMETER_TRANSFERS = "configure-task-transfers";
+
+  /**
+   * Replacements can be several per task, so they're passed in as a json-ified list of
+   * objects.
+   */
+  public static final String PARAMETER_REPLACEMENTS = "configure-task-replacements";
+
+  public static final String actionVerb = "Configuring";
+
+  public static final Float DEFAULT_PRIORITY = 1.0f;
+
+  /**
+   * Gson
+   */
+  private Gson m_gson = new Gson();
+
+  /**
+   * An optional brief description of config changes.
+   */
+  @XmlAttribute(name = "summary")
+  public String summary;
+
+  @XmlAttribute(name = "id", required = true)
+  public String id;
+
+  @XmlElement(name="type")
+  private String configType;
+
+  @XmlElement(name = "set")
+  private List<ConfigurationKeyValue> keyValuePairs;
+
+  @XmlElement(name = "condition")
+  private List<Condition> conditions;
+
+  @XmlElement(name = "transfer")
+  private List<Transfer> transfers;
+
+  @XmlElement(name="replace")
+  private List<Replace> replacements;
+
+  /**
+   * @return the config type
+   */
+  public String getConfigType() {
+    return configType;
+  }
+
+  /**
+   * @return the list of <set key=foo value=bar/> items
+   */
+  public List<ConfigurationKeyValue> getKeyValuePairs() {
+    return keyValuePairs;
+  }
+
+  /**
+   * @return the list of conditions
+   */
+  public List<Condition> getConditions() {
+    return conditions;
+  }
+
+  /**
+   * @return the list of transfers, checking for appropriate null fields.
+   */
+  public List<Transfer> getTransfers() {
+    if (null == transfers) {
+      return Collections.emptyList();
+    }
+
+    List<Transfer> list = new ArrayList<>();
+    for (Transfer t : transfers) {
+      switch (t.operation) {
+        case COPY:
+        case MOVE:
+          if (null != t.fromKey && null != t.toKey) {
+            list.add(t);
+          } else {
+            LOG.warn(String.format("Transfer %s is invalid", t));
+          }
+          break;
+        case DELETE:
+          if (null != t.deleteKey) {
+            list.add(t);
+          } else {
+            LOG.warn(String.format("Transfer %s is invalid", t));
+          }
+
+          break;
+      }
+    }
+
+    return list;
+  }
+
+  /**
+   * @return the replacement tokens, never {@code null}
+   */
+  public List<Replace> getReplacements() {
+    if (null == replacements) {
+      return Collections.emptyList();
+    }
+
+    List<Replace> list = new ArrayList<>();
+    for (Replace r : replacements) {
+      if (null == r.key || null == r.find || null == r.replaceWith) {
+        LOG.warn(String.format("Replacement %s is invalid", r));
+        continue;
+      }
+      list.add(r);
+    }
+
+    return list;
+  }
+
+  /**
+   * Used for configuration updates that should mask their values from being
+   * printed in plain text.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  public static class Masked {
+    @XmlAttribute(name = "mask")
+    public boolean mask = false;
+  }
+
+
+  /**
+   * A key/value pair to set in the type specified by {@link ConfigUpgradeChangeDefinition#configType}
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlType(name = "set")
+  public static class ConfigurationKeyValue extends Masked {
+    @XmlAttribute(name = "key")
+    public String key;
+
+    @XmlAttribute(name = "value")
+    public String value;
+  }
+
+  /**
+   * A conditional element that will only perform the configuration if the
+   * condition is met.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlType(name = "condition")
+  public static class Condition {
+    @XmlAttribute(name = "type")
+    private String conditionConfigType;
+
+    @XmlAttribute(name = "key")
+    private String conditionKey;
+
+    @XmlAttribute(name = "value")
+    private String conditionValue;
+
+    @XmlElement(name = "type")
+    private String configType;
+
+    @XmlElement(name = "key")
+    private String key;
+
+    @XmlElement(name = "value")
+    private String value;
+
+    public String getConditionConfigType() {
+      return conditionConfigType;
+    }
+
+    public String getConditionKey() {
+      return conditionKey;
+    }
+
+    public String getConditionValue() {
+      return conditionValue;
+    }
+
+    public String getConfigType() {
+      return configType;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    public String getValue() {
+      return value;
+    }
+  }
+
+  /**
+   * A {@code transfer} element will copy, move, or delete the value of one type/key to another type/key.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlType(name = "transfer")
+  public static class Transfer extends Masked {
+    /**
+     * The type of operation, such as COPY or DELETE.
+     */
+    @XmlAttribute(name = "operation")
+    public TransferOperation operation;
+
+    /**
+     * The configuration type to copy or move from.
+     */
+    @XmlAttribute(name = "from-type")
+    public String fromType;
+
+    /**
+     * The key to copy or move the configuration from.
+     */
+    @XmlAttribute(name = "from-key")
+    public String fromKey;
+
+    /**
+     * The key to copy the configuration value to.
+     */
+    @XmlAttribute(name = "to-key")
+    public String toKey;
+
+    /**
+     * The configuration key to delete, or "*" for all.
+     */
+    @XmlAttribute(name = "delete-key")
+    public String deleteKey;
+
+    /**
+     * If {@code true}, this will ensure that any changed properties are not
+     * removed during a {@link TransferOperation#DELETE}.
+     */
+    @XmlAttribute(name = "preserve-edits")
+    public boolean preserveEdits = false;
+
+    /**
+     * A default value to use when the configurations don't contain the
+     * {@link #fromKey}.
+     */
+    @XmlAttribute(name = "default-value")
+    public String defaultValue;
+
+    /**
+     * A data type to convert the configuration value to when the action is
+     * {@link TransferOperation#COPY}.
+     */
+    @XmlAttribute(name = "coerce-to")
+    public TransferCoercionType coerceTo;
+
+    // if the condition is true apply the transfer action
+    // only supported conditional action is DELETE
+    // if-type/if-key == if-value
+    /**
+     * The key to read for the if condition.
+     */
+    @XmlAttribute(name = "if-key")
+    public String ifKey;
+
+    /**
+     * The config type to read for the if condition.
+     */
+    @XmlAttribute(name = "if-type")
+    public String ifType;
+
+    /**
+     * The property value to compare against for the if condition.
+     */
+    @XmlAttribute(name = "if-value")
+    public String ifValue;
+
+    /**
+     * The keys to keep when the action is {@link TransferOperation#DELETE}.
+     */
+    @XmlElement(name = "keep-key")
+    public List<String> keepKeys = new ArrayList<String>();
+
+    @Override
+    public String toString() {
+      return "Transfer{" +
+              "operation=" + operation +
+              ", fromType='" + fromType + '\'' +
+              ", fromKey='" + fromKey + '\'' +
+              ", toKey='" + toKey + '\'' +
+              ", deleteKey='" + deleteKey + '\'' +
+              ", preserveEdits=" + preserveEdits +
+              ", defaultValue='" + defaultValue + '\'' +
+              ", coerceTo=" + coerceTo +
+              ", ifKey='" + ifKey + '\'' +
+              ", ifType='" + ifType + '\'' +
+              ", ifValue='" + ifValue + '\'' +
+              ", keepKeys=" + keepKeys +
+              '}';
+    }
+  }
+
+  /**
+   * Used to replace strings in a key with other strings.  More complex
+   * scenarios will be possible with regex (when needed)
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlType(name = "replace")
+  public static class Replace extends Masked {
+    /**
+     * The key name
+     */
+    @XmlAttribute(name="key")
+    public String key;
+
+    /**
+     * The string to find
+     */
+    @XmlAttribute(name="find")
+    public String find;
+
+    /**
+     * The string to replace
+     */
+    @XmlAttribute(name="replace-with")
+    public String replaceWith;
+
+    @Override
+    public String toString() {
+      return "Replace{" +
+              "key='" + key + '\'' +
+              ", find='" + find + '\'' +
+              ", replaceWith='" + replaceWith + '\'' +
+              '}';
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConfigureTask.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConfigureTask.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConfigureTask.java
index 8361ea6..a85c416 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConfigureTask.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConfigureTask.java
@@ -18,7 +18,6 @@
 package org.apache.ambari.server.state.stack.upgrade;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -26,11 +25,10 @@ import java.util.Map;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
-import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.XmlTransient;
 import javax.xml.bind.annotation.XmlType;
 
+import org.apache.ambari.server.state.stack.ConfigUpgradePack;
 import org.apache.commons.lang.StringUtils;
 import org.apache.ambari.server.serveraction.upgrades.ConfigureAction;
 import org.apache.ambari.server.state.Cluster;
@@ -40,41 +38,21 @@ import org.apache.ambari.server.state.DesiredConfig;
 import com.google.gson.Gson;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import static org.apache.ambari.server.state.stack.upgrade.ConfigUpgradeChangeDefinition.Transfer;
+import static org.apache.ambari.server.state.stack.upgrade.ConfigUpgradeChangeDefinition.Replace;
+import static org.apache.ambari.server.state.stack.upgrade.ConfigUpgradeChangeDefinition.Condition;
+import static org.apache.ambari.server.state.stack.upgrade.ConfigUpgradeChangeDefinition.ConfigurationKeyValue;
 
 /**
- * The {@link ConfigureTask} represents a configuration change. This task can be
- * defined with conditional statements that will only set values if a condition
- * passes:
+ * The {@link ConfigureTask} represents a configuration change. This task
+ * contains id of change. Change definitions are located in a separate file (config
+ * upgrade pack). IDs of change definitions share the same namespace within all
+ * stacks
  * <p/>
  *
  * <pre>
  * {@code
- * <task xsi:type="configure">
- *   <condition type="hive-site" key="hive.server2.transport.mode" value="binary">
- *     <type>hive-site</type>
- *     <key>hive.server2.thrift.port</key>
- *     <value>10010</value>
- *   </condition>
- *   <condition type="hive-site" key="hive.server2.transport.mode" value="http">
- *     <type>hive-site</type>
- *     <key>hive.server2.http.port</key>
- *     <value>10011</value>
- *   </condition>
- * </task>
- * }
- * </pre>
- *
- * It's also possible to simple set values directly without a precondition
- * check.
- *
- * <pre>
- * {@code
- * <task xsi:type="configure">
- *   <type>hive-site</type>
- *   <set key="hive.server2.thrift.port" value="10010"/>
- *   <set key="foo" value="bar"/>
- *   <set key="foobar" value="baz"/>
- * </task>
+ * <task xsi:type="configure" id="hdp_2_3_0_0-UpdateHiveConfig"/>
  * }
  * </pre>
  *
@@ -118,29 +96,15 @@ public class ConfigureTask extends ServerSideActionTask {
 
   /**
    * Constructor.
-   *
    */
   public ConfigureTask() {
     implClass = ConfigureAction.class.getName();
   }
 
-  @XmlTransient
   private Task.Type type = Task.Type.CONFIGURE;
 
-  @XmlElement(name="type")
-  private String configType;
-
-  @XmlElement(name = "set")
-  private List<ConfigurationKeyValue> keyValuePairs;
-
-  @XmlElement(name = "condition")
-  private List<Condition> conditions;
-
-  @XmlElement(name = "transfer")
-  private List<Transfer> transfers;
-
-  @XmlElement(name="replace")
-  private List<Replace> replacements;
+  @XmlAttribute(name = "id")
+  public String id;
 
   /**
    * {@inheritDoc}
@@ -161,222 +125,6 @@ public class ConfigureTask extends ServerSideActionTask {
   }
 
   /**
-   * @return the config type
-   */
-  public String getConfigType() {
-    return configType;
-  }
-
-  /**
-   * Used for configuration updates that should mask their values from being
-   * printed in plain text.
-   */
-  @XmlAccessorType(XmlAccessType.FIELD)
-  public static class Masked {
-    @XmlAttribute(name = "mask")
-    public boolean mask = false;
-  }
-
-
-  /**
-   * A key/value pair to set in the type specified by {@link ConfigureTask#type}
-   */
-  @XmlAccessorType(XmlAccessType.FIELD)
-  @XmlType(name = "set")
-  public static class ConfigurationKeyValue extends Masked {
-    @XmlAttribute(name = "key")
-    public String key;
-
-    @XmlAttribute(name = "value")
-    public String value;
-  }
-
-  /**
-   * A conditional element that will only perform the configuration if the
-   * condition is met.
-   */
-  @XmlAccessorType(XmlAccessType.FIELD)
-  @XmlType(name = "condition")
-  public static class Condition {
-    @XmlAttribute(name = "type")
-    private String conditionConfigType;
-
-    @XmlAttribute(name = "key")
-    private String conditionKey;
-
-    @XmlAttribute(name = "value")
-    private String conditionValue;
-
-    @XmlElement(name = "type")
-    private String configType;
-
-    @XmlElement(name = "key")
-    private String key;
-
-    @XmlElement(name = "value")
-    private String value;
-  }
-
-  /**
-   * A {@code transfer} element will copy, move, or delete the value of one type/key to another type/key.
-   */
-  @XmlAccessorType(XmlAccessType.FIELD)
-  @XmlType(name = "transfer")
-  public static class Transfer extends Masked {
-    /**
-     * The type of operation, such as COPY or DELETE.
-     */
-    @XmlAttribute(name = "operation")
-    public TransferOperation operation;
-
-    /**
-     * The configuration type to copy or move from.
-     */
-    @XmlAttribute(name = "from-type")
-    public String fromType;
-
-    /**
-     * The key to copy or move the configuration from.
-     */
-    @XmlAttribute(name = "from-key")
-    public String fromKey;
-
-    /**
-     * The key to copy the configuration value to.
-     */
-    @XmlAttribute(name = "to-key")
-    public String toKey;
-
-    /**
-     * The configuration key to delete, or "*" for all.
-     */
-    @XmlAttribute(name = "delete-key")
-    public String deleteKey;
-
-    /**
-     * If {@code true}, this will ensure that any changed properties are not
-     * removed during a {@link TransferOperation#DELETE}.
-     */
-    @XmlAttribute(name = "preserve-edits")
-    public boolean preserveEdits = false;
-
-    /**
-     * A default value to use when the configurations don't contain the
-     * {@link #fromKey}.
-     */
-    @XmlAttribute(name = "default-value")
-    public String defaultValue;
-
-    /**
-     * A data type to convert the configuration value to when the action is
-     * {@link TransferOperation#COPY}.
-     */
-    @XmlAttribute(name = "coerce-to")
-    public TransferCoercionType coerceTo;
-
-    // if the condition is true apply the transfer action
-    // only supported conditional action is DELETE
-    // if-type/if-key == if-value
-    /**
-     * The key to read for the if condition.
-     */
-    @XmlAttribute(name = "if-key")
-    public String ifKey;
-
-    /**
-     * The config type to read for the if condition.
-     */
-    @XmlAttribute(name = "if-type")
-    public String ifType;
-
-    /**
-     * The property value to compare against for the if condition.
-     */
-    @XmlAttribute(name = "if-value")
-    public String ifValue;
-
-    /**
-     * The keys to keep when the action is {@link TransferOperation#DELETE}.
-     */
-    @XmlElement(name = "keep-key")
-    public List<String> keepKeys = new ArrayList<String>();
-  }
-
-  /**
-   * @return the list of transfers, checking for appropriate null fields.
-   */
-  public List<Transfer> getTransfers() {
-    if (null == transfers) {
-      return Collections.<Transfer>emptyList();
-    }
-
-    List<Transfer> list = new ArrayList<Transfer>();
-    for (Transfer t : transfers) {
-      switch (t.operation) {
-        case COPY:
-        case MOVE:
-          if (null != t.fromKey && null != t.toKey) {
-            list.add(t);
-          }
-          break;
-        case DELETE:
-          if (null != t.deleteKey) {
-            list.add(t);
-          }
-
-          break;
-      }
-    }
-
-    return list;
-  }
-
-  /**
-   * Used to replace strings in a key with other strings.  More complex
-   * scenarios will be possible with regex (when needed)
-   */
-  @XmlAccessorType(XmlAccessType.FIELD)
-  @XmlType(name = "replace")
-  public static class Replace extends Masked {
-    /**
-     * The key name
-     */
-    @XmlAttribute(name="key")
-    public String key;
-
-    /**
-     * The string to find
-     */
-    @XmlAttribute(name="find")
-    public String find;
-
-    /**
-     * The string to replace
-     */
-    @XmlAttribute(name="replace-with")
-    public String replaceWith;
-  }
-
-  /**
-   * @return the replacement tokens, never {@code null}
-   */
-  public List<Replace> getReplacements() {
-    if (null == replacements) {
-      return Collections.emptyList();
-    }
-
-    List<Replace> list = new ArrayList<Replace>();
-    for (Replace r : replacements) {
-      if (null == r.key || null == r.find || null == r.replaceWith) {
-        continue;
-      }
-      list.add(r);
-    }
-
-    return list;
-  }
-
-  /**
    * Gets a map containing the following properties pertaining to the
    * configuration value to change:
    * <ul>
@@ -397,21 +145,41 @@ public class ConfigureTask extends ServerSideActionTask {
    *         handle a configuration task that is unable to set any configuration
    *         values.
    */
-  public Map<String, String> getConfigurationChanges(Cluster cluster) {
-    Map<String, String> configParameters = new HashMap<String, String>();
+  public Map<String, String> getConfigurationChanges(Cluster cluster,
+                                                     ConfigUpgradePack configUpgradePack) {
+    Map<String, String> configParameters = new HashMap<>();
+
+    if (this.id == null || this.id.isEmpty()) {
+      LOG.warn("Config task id is not defined, skipping config change");
+      return configParameters;
+    }
+
+    if (configUpgradePack == null) {
+      LOG.warn("Config upgrade pack is not defined, skipping config change");
+      return configParameters;
+    }
+
+    // extract config change definition, referenced by current ConfigureTask
+    ConfigUpgradeChangeDefinition definition = configUpgradePack.enumerateConfigChangesByID().get(this.id);
+    if (definition == null) {
+      LOG.warn(String.format("Can not resolve config change definition by id %s, " +
+              "skipping config change", this.id));
+      return configParameters;
+    }
 
     // the first matched condition will win; conditions make configuration tasks singular in
     // the properties that can be set - when there is a condition the task will only contain
     // conditions
+    List<Condition> conditions = definition.getConditions();
     if( null != conditions && !conditions.isEmpty() ){
       for (Condition condition : conditions) {
-        String conditionConfigType = condition.conditionConfigType;
-        String conditionKey = condition.conditionKey;
-        String conditionValue = condition.conditionValue;
+        String conditionConfigType = condition.getConditionConfigType();
+        String conditionKey = condition.getConditionKey();
+        String conditionValue = condition.getConditionValue();
 
         // always add the condition's target type just so that we have one to
         // return even if none of the conditions match
-        configParameters.put(PARAMETER_CONFIG_TYPE, condition.configType);
+        configParameters.put(PARAMETER_CONFIG_TYPE, condition.getConfigType());
 
         // check the condition; if it passes, set the configuration properties
         // and break
@@ -419,10 +187,10 @@ public class ConfigureTask extends ServerSideActionTask {
             conditionConfigType, conditionKey);
 
         if (conditionValue.equals(checkValue)) {
-          List<ConfigurationKeyValue> configurations = new ArrayList<ConfigurationKeyValue>(1);
+          List<ConfigurationKeyValue> configurations = new ArrayList<>(1);
           ConfigurationKeyValue keyValue = new ConfigurationKeyValue();
-          keyValue.key = condition.key;
-          keyValue.value = condition.value;
+          keyValue.key = condition.getKey();
+          keyValue.value = condition.getValue();
           configurations.add(keyValue);
 
           configParameters.put(ConfigureTask.PARAMETER_KEY_VALUE_PAIRS,
@@ -434,20 +202,21 @@ public class ConfigureTask extends ServerSideActionTask {
     }
 
     // this task is not a condition task, so process the other elements normally
-    if (null != configType) {
-      configParameters.put(PARAMETER_CONFIG_TYPE, configType);
+    if (null != definition.getConfigType()) {
+      configParameters.put(PARAMETER_CONFIG_TYPE, definition.getConfigType());
     }
 
     // for every <set key=foo value=bar/> add it to this list
-    if (null != keyValuePairs && !keyValuePairs.isEmpty()) {
+    if (null != definition.getKeyValuePairs() && !definition.getKeyValuePairs().isEmpty()) {
       configParameters.put(ConfigureTask.PARAMETER_KEY_VALUE_PAIRS,
-          m_gson.toJson(keyValuePairs));
+          m_gson.toJson(definition.getKeyValuePairs()));
     }
 
     // transfers
+    List<Transfer> transfers = definition.getTransfers();
     if (null != transfers && !transfers.isEmpty()) {
 
-      List<Transfer> allowedTransfers = new ArrayList<Transfer>();
+      List<Transfer> allowedTransfers = new ArrayList<>();
       for (Transfer transfer : transfers) {
         if (transfer.operation == TransferOperation.DELETE) {
           if (StringUtils.isNotBlank(transfer.ifKey) &&
@@ -462,7 +231,7 @@ public class ConfigureTask extends ServerSideActionTask {
             if (!ifValue.toLowerCase().equals(StringUtils.lowerCase(checkValue))) {
               // skip adding
               LOG.info("Skipping property delete for {}/{} as the value {} for {}/{} is not equal to {}",
-                       this.getConfigType(), transfer.deleteKey, checkValue, ifConfigType, ifKey, ifValue);
+                       definition.getConfigType(), transfer.deleteKey, checkValue, ifConfigType, ifKey, ifValue);
               continue;
             }
           }
@@ -473,6 +242,7 @@ public class ConfigureTask extends ServerSideActionTask {
     }
 
     // replacements
+    List<Replace> replacements = definition.getReplacements();
     if( null != replacements && !replacements.isEmpty() ){
       configParameters.put(ConfigureTask.PARAMETER_REPLACEMENTS, m_gson.toJson(replacements));
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Grouping.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Grouping.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Grouping.java
index 9d89b7a..ec0fabf 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Grouping.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Grouping.java
@@ -83,7 +83,7 @@ public class Grouping {
      * E.g., preupgrade, restart hosts(0), ..., restart hosts(n-1), postupgrade
      * @param hostsType the order collection of hosts, which may have a master and secondary
      * @param service the service name
-     * @param pc the ProcessingComponent derived from the upgrade pack.
+     * @param pc the AffectedComponent derived from the upgrade pack.
      */
     @Override
     public void add(UpgradeContext ctx, HostsType hostsType, String service,

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/StageWrapperBuilder.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/StageWrapperBuilder.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/StageWrapperBuilder.java
index c9c6b8c..6c0f3c7 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/StageWrapperBuilder.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/StageWrapperBuilder.java
@@ -37,7 +37,7 @@ public abstract class StageWrapperBuilder {
    * @param hostsType   the hosts, along with their type
    * @param service     the service name
    * @param clientOnly  whether the service is client only, no service checks
-   * @param pc          the ProcessingComponent derived from the upgrade pack
+   * @param pc          the AffectedComponent derived from the upgrade pack
    */
   public abstract void add(UpgradeContext ctx, HostsType hostsType, String service,
       boolean clientOnly, ProcessingComponent pc);

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/config-upgrade.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/config-upgrade.xml b/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/config-upgrade.xml
new file mode 100644
index 0000000..fbd21a4
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/config-upgrade.xml
@@ -0,0 +1,55 @@
+<?xml version="1.0"?>
+<!--
+   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.
+-->
+
+<upgrade-config-changes xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+  <services>
+    <service name="HIVE">
+      <component name="HIVE_SERVER">
+        <changes>
+          <definition xsi:type="configure" id="hdp_2_2_0_0_hive_server_set_transport_modes">
+            <condition type="hive-site" key="hive.server2.transport.mode" value="binary">
+              <type>hive-site</type>
+              <key>hive.server2.thrift.port</key>
+              <value>10010</value>
+            </condition>
+            <condition type="hive-site" key="hive.server2.transport.mode" value="http">
+              <type>hive-site</type>
+              <key>hive.server2.http.port</key>
+              <value>10011</value>
+            </condition>
+          </definition>
+
+          <definition xsi:type="configure" id="hdp_2_2_0_0_hive_server_restore_transport_mode_on_downgrade">
+            <condition type="hive-site" key="hive.server2.transport.mode" value="binary">
+              <type>hive-site</type>
+              <key>hive.server2.thrift.port</key>
+              <value>10000</value>
+            </condition>
+            <condition type="hive-site" key="hive.server2.transport.mode" value="http">
+              <type>hive-site</type>
+              <key>hive.server2.http.port</key>
+              <value>10001</value>
+            </condition>
+          </definition>
+        </changes>
+      </component>
+    </service>
+  </services>
+
+</upgrade-config-changes>

http://git-wip-us.apache.org/repos/asf/ambari/blob/a67ddd27/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml b/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml
index 74eb499..9900d6a 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml
@@ -464,18 +464,7 @@
             <message>Please note that the HiveServer port will now change to 10010 if hive is using a binary transfer mode or 10011 if hive is using an http transport mode. You can use "netstat -anp | grep 1001[01]" to determine if the port is available on each of following HiveServer host(s): {{hosts.all}}. If the port is not available, the process using it must be terminated.</message>
           </task>
 
-          <task xsi:type="configure">
-            <condition type="hive-site" key="hive.server2.transport.mode" value="binary">
-              <type>hive-site</type>
-              <key>hive.server2.thrift.port</key>
-              <value>10010</value>
-            </condition>
-            <condition type="hive-site" key="hive.server2.transport.mode" value="http">
-              <type>hive-site</type>
-              <key>hive.server2.http.port</key>
-              <value>10011</value>
-            </condition>
-          </task>
+          <task xsi:type="configure" id="hdp_2_2_0_0_hive_server_set_transport_modes" />
         </pre-upgrade>
 
         <pre-downgrade>
@@ -484,18 +473,7 @@
             <message>Please note that the HiveServer port will now change to 10000 if hive is using a binary transfer mode or 10001 if hive is using an http transport mode. You can use "netstat -anp | grep 1000[01]" to determine if the port is available on each of following HiveServer host(s): {{hosts.all}}. If the port is not available, the process using it must be terminated.</message>
           </task>
 
-          <task xsi:type="configure">
-            <condition type="hive-site" key="hive.server2.transport.mode" value="binary">
-              <type>hive-site</type>
-              <key>hive.server2.thrift.port</key>
-              <value>10000</value>
-            </condition>
-            <condition type="hive-site" key="hive.server2.transport.mode" value="http">
-              <type>hive-site</type>
-              <key>hive.server2.http.port</key>
-              <value>10001</value>
-            </condition>
-          </task>
+          <task xsi:type="configure" id="hdp_2_2_0_0_hive_server_restore_transport_mode_on_downgrade" />
         </pre-downgrade>
 
         <upgrade>