You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by nc...@apache.org on 2014/11/10 15:40:02 UTC

ambari git commit: AMBARI-8206. Upgrade Execute: Load and parse upgrade packs from stack directory (ncole)

Repository: ambari
Updated Branches:
  refs/heads/trunk 5c6bfc765 -> b17637c9b


AMBARI-8206. Upgrade Execute: Load and parse upgrade packs from stack directory (ncole)


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

Branch: refs/heads/trunk
Commit: b17637c9b8b42594f92a017cf0846589251917f1
Parents: 5c6bfc7
Author: Nate Cole <nc...@hortonworks.com>
Authored: Fri Nov 7 10:22:47 2014 -0500
Committer: Nate Cole <nc...@hortonworks.com>
Committed: Mon Nov 10 09:39:42 2014 -0500

----------------------------------------------------------------------
 .../server/api/services/AmbariMetaInfo.java     |  62 ++++++
 .../server/api/util/StackExtensionHelper.java   | 137 ++++++-------
 .../ambari/server/controller/AmbariServer.java  |   1 -
 .../apache/ambari/server/state/StackInfo.java   |  27 ++-
 .../ambari/server/state/stack/UpgradePack.java  | 159 +++++++++++++++
 .../server/state/stack/upgrade/Batch.java       |  51 +++++
 .../state/stack/upgrade/ConditionalBatch.java   |  46 +++++
 .../state/stack/upgrade/ConfigureTask.java      |  47 +++++
 .../server/state/stack/upgrade/CountBatch.java  |  42 ++++
 .../server/state/stack/upgrade/ExecuteTask.java |  41 ++++
 .../server/state/stack/upgrade/ManualTask.java  |  42 ++++
 .../state/stack/upgrade/PercentBatch.java       |  43 ++++
 .../ambari/server/state/stack/upgrade/Task.java |  54 +++++
 .../stacks/HDP/2.2/upgrades/upgrade-2.2.xml     |  54 +++++
 .../server/state/stack/UpgradePackTest.java     | 198 +++++++++++++++++++
 .../stacks/HDP/2.1.1/upgrades/upgrade_test.xml  |  83 ++++++++
 16 files changed, 1015 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/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 bb4c569..ba365f5 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
@@ -73,6 +73,8 @@ import org.apache.ambari.server.state.stack.OsFamily;
 import org.apache.ambari.server.state.stack.RepositoryXml;
 import org.apache.ambari.server.state.stack.RepositoryXml.Os;
 import org.apache.ambari.server.state.stack.RepositoryXml.Repo;
+import org.apache.ambari.server.state.stack.UpgradePack;
+import org.apache.commons.io.FilenameUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -97,6 +99,7 @@ public class AmbariMetaInfo {
   public static final String RCO_FILE_NAME = "role_command_order.json";
   public static final String SERVICE_METRIC_FILE_NAME = "metrics.json";
   public static final String SERVICE_ALERT_FILE_NAME = "alerts.json";
+
   /**
    * This string is used in placeholder in places that are common for
    * all operating systems or in situations where os type is not important.
@@ -121,6 +124,8 @@ public class AmbariMetaInfo {
   private static final String REPOSITORY_FILE_NAME = "repoinfo.xml";
   private static final String REPOSITORY_FOLDER_NAME = "repos";
   public static final String REPOSITORY_XML_PROPERTY_BASEURL = "baseurl";
+  private static final String UPGRADE_PACK_FOLDER_NAME = "upgrades";
+
   // all the supported OS'es
   @Inject
   private OsFamily os_family;
@@ -967,6 +972,11 @@ public class AmbariMetaInfo {
       // Resolve hooks folder
       String stackHooksToUse = stackExtensionHelper.resolveHooksFolder(stack);
       stack.setStackHooksFolder(stackHooksToUse);
+
+      File upgradesFolder = new File(stackPath + File.separator + UPGRADE_PACK_FOLDER_NAME);
+      if (upgradesFolder.exists() && upgradesFolder.isDirectory()) {
+        stack.setUpgradesFolder(upgradesFolder.getAbsolutePath());
+      }
     }
 
     es.invokeAll(lookupList);
@@ -1345,4 +1355,56 @@ public class AmbariMetaInfo {
       }
     }
   }
+
+  /**
+   * Gets upgrade packs available for a stack.
+   * @param stackName the stack name
+   * @param stackVersion the stack version
+   * @return a map of upgrade packs, keyed by the name of the upgrade pack
+   */
+  public Map<String, UpgradePack> getUpgradePacks(String stackName, String stackVersion) {
+    StackInfo stack = null;
+    try {
+      stack = getStackInfo(stackName, stackVersion);
+    } catch (AmbariException e) {
+      LOG.debug("Cannot load upgrade packs for non-existent stack {}-{}",
+          stackName, stackVersion, e);
+      return Collections.emptyMap();
+    }
+
+    File folder = new File(stack.getUpgradesFolder());
+    if (!folder.exists() || !folder.isDirectory()) {
+      LOG.error("Upgrades folder {} no longer exists", stack.getUpgradesFolder());
+      return Collections.emptyMap();
+    }
+
+    String[] fileNames = folder.list(new FilenameFilter() {
+      @Override
+      public boolean accept(File folder, String fileName) {
+        if (fileName.toLowerCase().endsWith(".xml")) {
+          return true;
+        } else {
+          return false;
+        }
+      }
+    });
+
+
+    Map<String, UpgradePack> packs = new HashMap<String, UpgradePack>();
+
+    for (String fileName : fileNames) {
+      File f = new File(folder, fileName);
+
+      String packName = FilenameUtils.removeExtension(fileName);
+
+      try {
+        UpgradePack up = StackExtensionHelper.unmarshal(UpgradePack.class, f);
+        packs.put(packName, up);
+      } catch (Exception e) {
+        LOG.error("Could not parse {} into an upgrade pack", f.getAbsolutePath());
+      }
+    }
+
+    return packs;
+  }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/main/java/org/apache/ambari/server/api/util/StackExtensionHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/api/util/StackExtensionHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/api/util/StackExtensionHelper.java
index 781ea13..e014627 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/api/util/StackExtensionHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/api/util/StackExtensionHelper.java
@@ -17,7 +17,32 @@
  */
 package org.apache.ambari.server.api.util;
 
-import com.google.inject.Injector;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.namespace.QName;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathExpression;
+import javax.xml.xpath.XPathExpressionException;
+import javax.xml.xpath.XPathFactory;
+
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.metadata.ActionMetadata;
@@ -35,35 +60,13 @@ import org.apache.ambari.server.state.stack.ConfigurationXml;
 import org.apache.ambari.server.state.stack.RepositoryXml;
 import org.apache.ambari.server.state.stack.ServiceMetainfoXml;
 import org.apache.ambari.server.state.stack.StackMetainfoXml;
+import org.apache.ambari.server.state.stack.UpgradePack;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
 import org.xml.sax.SAXException;
 
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.namespace.QName;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathExpression;
-import javax.xml.xpath.XPathExpressionException;
-import javax.xml.xpath.XPathFactory;
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.Set;
+import com.google.inject.Injector;
 
 /**
  * Helper methods for providing stack extension behavior -
@@ -79,7 +82,7 @@ import java.util.Set;
  */
 public class StackExtensionHelper {
   private ActionMetadata actionMetadata;
-  
+
   private File stackRoot;
   private final static Logger LOG = LoggerFactory.getLogger(StackExtensionHelper.class);
   private final Map<String, StackInfo> stackVersionMap = new HashMap<String,
@@ -92,11 +95,12 @@ public class StackExtensionHelper {
       new HashMap<Class<?>, JAXBContext> ();
   static {
     try {
-      // three classes define the top-level element "metainfo", so we need 3 contexts.
-      JAXBContext ctx = JAXBContext.newInstance(StackMetainfoXml.class, RepositoryXml.class, ConfigurationXml.class);
+      JAXBContext ctx = JAXBContext.newInstance(StackMetainfoXml.class,
+          RepositoryXml.class, ConfigurationXml.class, UpgradePack.class);
       _jaxbContexts.put(StackMetainfoXml.class, ctx);
       _jaxbContexts.put(RepositoryXml.class, ctx);
       _jaxbContexts.put(ConfigurationXml.class, ctx);
+      _jaxbContexts.put(UpgradePack.class, ctx);
       _jaxbContexts.put(ServiceMetainfoXml.class, JAXBContext.newInstance(ServiceMetainfoXml.class));
     } catch (JAXBException e) {
       throw new RuntimeException (e);
@@ -180,9 +184,9 @@ public class StackExtensionHelper {
             parentService.getRequiredServices() :
             Collections.<String>emptyList()
       );
-    
+
     mergedServiceInfo.setRestartRequiredAfterChange(
-            (childService.isRestartRequiredAfterChange() != null) 
+            (childService.isRestartRequiredAfterChange() != null)
                     ? childService.isRestartRequiredAfterChange()
                     : parentService.isRestartRequiredAfterChange());
     mergedServiceInfo.setMonitoringService(
@@ -196,7 +200,7 @@ public class StackExtensionHelper {
     } else {
       mergedServiceInfo.setOsSpecifics(parentService.getOsSpecifics());
     }
-    
+
     mergedServiceInfo.setConfigTypes(new HashMap<String, Map<String, Map<String, String>>>());
     if(childService.getConfigTypes() != null) {
       mergedServiceInfo.getConfigTypes().putAll(childService.getConfigTypes());
@@ -204,14 +208,14 @@ public class StackExtensionHelper {
     if(parentService.getConfigTypes() != null) {
       mergedServiceInfo.getConfigTypes().putAll(parentService.getConfigTypes());
     }
-    
+
     CommandScriptDefinition commandScript = childService.getCommandScript();
     if (commandScript != null) {
        mergedServiceInfo.setCommandScript(childService.getCommandScript());
     } else {
       mergedServiceInfo.setCommandScript(parentService.getCommandScript());
     }
-    
+
     String servicePackageFolder = childService.getServicePackageFolder();
     if (servicePackageFolder != null) {
       mergedServiceInfo.setServicePackageFolder(servicePackageFolder);
@@ -225,13 +229,13 @@ public class StackExtensionHelper {
             mergeCustomCommandLists(parentService.getCustomCommands(),
                     childService.getCustomCommands());
     mergedServiceInfo.setCustomCommands(mergedCustomCommands);
-    
+
     // metrics
     if (null != childService.getMetricsFile())
       mergedServiceInfo.setMetricsFile(childService.getMetricsFile());
     else if (null != parentService.getMetricsFile())
       mergedServiceInfo.setMetricsFile(parentService.getMetricsFile());
-    
+
     // alerts
     if (null != childService.getAlertsFile())
       mergedServiceInfo.setAlertsFile(childService.getAlertsFile());
@@ -252,8 +256,8 @@ public class StackExtensionHelper {
     }
     return mergedServiceInfo;
   }
-  
-  public void mergeProperties(List<PropertyInfo> childProperties, 
+
+  public void mergeProperties(List<PropertyInfo> childProperties,
                               List<PropertyInfo> parentProperties, List<PropertyInfo> mergedProperties, Map<String,
       Map<String, Map<String, String>>> childConfigTypes) {
     // Add child properties not deleted
@@ -464,9 +468,9 @@ public class StackExtensionHelper {
       }
       StackInfo currentStackInfo = lt.previous();
       serviceInfoList = currentStackInfo.getServices();
-      
+
       mergeStacks(parentStack, currentStackInfo);
-      
+
       for (ServiceInfo service : serviceInfoList) {
         ServiceInfo existingService = serviceInfoMap.get(service.getName());
         if (service.isDeleted()) {
@@ -481,21 +485,21 @@ public class StackExtensionHelper {
           ServiceInfo newServiceInfo = mergeServices(existingService, service);
           serviceInfoMap.put(service.getName(), newServiceInfo);
         }
-        
+
         // remove 'excluded-config-types' from configTypes
         ServiceInfo serviceInfo = serviceInfoMap.get(service.getName());
-        if(serviceInfo.getExcludedConfigTypes() != null) { 
+        if(serviceInfo.getExcludedConfigTypes() != null) {
           Iterator<Map.Entry<String,Map<String,Map<String,String>>>> configTypesItetator = serviceInfo.getConfigTypes().entrySet().iterator();
-          
+
           while(configTypesItetator.hasNext()) {
             Map.Entry<String,Map<String,Map<String,String>>> configTypeMap = configTypesItetator.next();
-            
+
             if(serviceInfo.getExcludedConfigTypes().contains(configTypeMap.getKey())) {
               configTypesItetator.remove();
             }
           }
         }
-        
+
       }
       parentStack = currentStackInfo;
     }
@@ -563,10 +567,10 @@ public class StackExtensionHelper {
           // get metrics file, if it exists
           File metricsJson = new File(serviceFolder.getAbsolutePath()
             + File.separator + AmbariMetaInfo.SERVICE_METRIC_FILE_NAME);
-          
+
           File alertsJson = new File(serviceFolder.getAbsolutePath() +
               File.separator + AmbariMetaInfo.SERVICE_ALERT_FILE_NAME);
-          
+
           if (!metainfoFile.exists()) {
             LOG.warn("Service folder " + serviceFolder.getAbsolutePath() + " doesn't contain metainfo file. Ignoring it.");
             continue;
@@ -738,9 +742,9 @@ public class StackExtensionHelper {
         LOG.debug("Reading stack version metainfo from file "
             + stackMetainfoFile.getAbsolutePath());
       }
-      
+
       StackMetainfoXml smx = unmarshal(StackMetainfoXml.class, stackMetainfoFile);
-      
+
       stackInfo.setMinUpgradeVersion(smx.getVersion().getUpgrade());
       stackInfo.setActive(smx.getVersion().isActive());
       stackInfo.setParentStackVersion(smx.getExtends());
@@ -762,7 +766,7 @@ public class StackExtensionHelper {
               File.separator + AmbariMetaInfo.RCO_FILE_NAME;
       if (new File(rcoFileLocation).exists())
         stackInfo.setRcoFileLocation(rcoFileLocation);
-      
+
       setStackPropertiesFromConfigs(stackInfo);
     }
 
@@ -776,7 +780,7 @@ public class StackExtensionHelper {
     }
     return stackInfo;
   }
-  
+
   private void populateStackProperties(StackInfo stackInfo, File configFile) throws JAXBException {
     ConfigurationXml configuration = unmarshal(ConfigurationXml.class, configFile);
     String fileName = configFile.getName();
@@ -786,7 +790,7 @@ public class StackExtensionHelper {
     addConfigType(stackInfo.getConfigTypes(), configType);
     setConfigTypeAttributes(stackInfo.getConfigTypes(), configuration, configType);
   }
-  
+
   /**
    * Get all properties from all "configs/*.xml" files. See {@see AmbariMetaInfo#SERVICE_CONFIG_FILE_NAME_POSTFIX}
    */
@@ -794,10 +798,10 @@ public class StackExtensionHelper {
     File configsFolder = new File(stackRoot.getAbsolutePath() + File
         .separator + stackInfo.getName() + File.separator + stackInfo.getVersion()
         + File.separator + AmbariMetaInfo.SERVICE_CONFIG_FOLDER_NAME);
-    
+
     if (!configsFolder.exists() || !configsFolder.isDirectory())
       return;
-    
+
     File[] configFiles = configsFolder.listFiles(AmbariMetaInfo.FILENAME_FILTER);
     if (configFiles != null) {
       for (File configFile : configFiles) {
@@ -833,7 +837,7 @@ public class StackExtensionHelper {
     addConfigType(serviceInfo.getConfigTypes(), configType);
     setConfigTypeAttributes(serviceInfo.getConfigTypes(), configuration, configType);
   }
-  
+
   void setConfigTypeAttributes(Map<String, Map<String, Map<String, String>>> configTypes, ConfigurationXml configuration, String configType) {
     for (Map.Entry<QName, String> attribute : configuration.getAttributes().entrySet()) {
       for (Supports supportsProperty : Supports.values()) {
@@ -846,16 +850,16 @@ public class StackExtensionHelper {
       }
     }
   }
-  
+
   void addConfigType(Map<String, Map<String, Map<String, String>>> configTypes, String configType) {
     configTypes.put(configType, new HashMap<String, Map<String, String>>());
-    
+
     Map<String, Map<String, String>> properties = configTypes.get(configType);
     Map<String, String> supportsProperties = new HashMap<String, String>();
     for (Supports supportsProperty : Supports.values()) {
       supportsProperties.put(supportsProperty.getPropertyName(), supportsProperty.getDefaultValue());
     }
-    properties.put(Supports.KEYWORD, supportsProperties); 
+    properties.put(Supports.KEYWORD, supportsProperties);
   }
 
   /**
@@ -867,7 +871,7 @@ public class StackExtensionHelper {
       Map<String, Map<String, Map<String, String>>> configTypes = new HashMap<String, Map<String, Map<String, String>>>();
       for (PropertyInfo configuration : configurations) {
         String configType = ConfigHelper.fileNameToConfigType(configuration.getFilename());
-        
+
         if (!configTypes.containsKey(configType)) {
           Map<String, Map<String, String>> properties = new HashMap<String, Map<String, String>>();
           Map<String, String> supportsProperties = new HashMap<String, String>();
@@ -901,13 +905,13 @@ public class StackExtensionHelper {
    * Get all properties from all "configs/*.xml" files. See {@see AmbariMetaInfo#SERVICE_CONFIG_FILE_NAME_POSTFIX}
    */
   void setPropertiesFromConfigs(File serviceFolder, ServiceInfo serviceInfo) {
-    
+
     File serviceConfigFolder = new File(serviceFolder.getAbsolutePath()
             + File.separator + serviceInfo.getConfigDir());
-    
+
     if (!serviceConfigFolder.exists() || !serviceConfigFolder.isDirectory())
       return;
-    
+
     File[] configFiles = serviceConfigFolder.listFiles(AmbariMetaInfo.FILENAME_FILTER);
     if (configFiles != null) {
       for (File configFile : configFiles) {
@@ -921,17 +925,17 @@ public class StackExtensionHelper {
       }
     }
   }
-  
+
   public static <T> T unmarshal(Class<T> clz, File file) throws JAXBException {
     Unmarshaller u = _jaxbContexts.get(clz).createUnmarshaller();
-    
+
     return clz.cast(u.unmarshal(file));
-  }  
-  
+  }
+
   /**
    * Service configuration-types can support different abilities. This
    * enumerates the various abilities that configuration-types can support.
-   * 
+   *
    * For example, Hadoop configuration types like 'core-site' and 'hdfs-site'
    * can support the ability to define certain configs as 'final'.
    */
@@ -967,4 +971,5 @@ public class StackExtensionHelper {
       return name().toLowerCase();
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariServer.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariServer.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariServer.java
index 224cce4..d21f81b 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariServer.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariServer.java
@@ -29,7 +29,6 @@ import javax.crypto.BadPaddingException;
 
 import org.apache.ambari.eventdb.webservice.WorkflowJsonService;
 import org.apache.ambari.server.AmbariException;
-import org.apache.ambari.server.StaticallyInject;
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.agent.HeartBeatHandler;
 import org.apache.ambari.server.agent.rest.AgentResource;

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/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 242cfcc..64782cc 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
@@ -44,6 +44,8 @@ public class StackInfo implements Comparable<StackInfo>{
    */
   private String stackHooksFolder;
 
+  private String upgradesFolder = null;
+
   public String getName() {
     return name;
   }
@@ -77,7 +79,7 @@ public class StackInfo implements Comparable<StackInfo>{
   public synchronized void setServices(List<ServiceInfo> services) {
     this.services = services;
   }
-  
+
   public List<PropertyInfo> getProperties() {
     if (properties == null) properties = new ArrayList<PropertyInfo>();
     return properties;
@@ -86,7 +88,7 @@ public class StackInfo implements Comparable<StackInfo>{
   public void setProperties(List<PropertyInfo> properties) {
     this.properties = properties;
   }
-  
+
   public Map<String, Map<String, Map<String, String>>> getConfigTypes() {
     if (configTypes == null) configTypes = new HashMap<String, Map<String, Map<String, String>>>();
     return configTypes;
@@ -117,15 +119,15 @@ public class StackInfo implements Comparable<StackInfo>{
 
     return sb.toString();
   }
-  
-  
+
+
   @Override
   public int hashCode() {
     int result = 1;
     result = 31  + name.hashCode() + version.hashCode();
     return result;
   }
-  
+
   @Override
   public boolean equals(Object obj) {
     if (!(obj instanceof StackInfo)) {
@@ -190,4 +192,19 @@ public class StackInfo implements Comparable<StackInfo>{
     String oId = o.name + "-" + o.version;
     return myId.compareTo(oId);
   }
+
+  /**
+   * @param path the path to the upgrades folder
+   */
+  public void setUpgradesFolder(String path) {
+    upgradesFolder = path;
+  }
+
+  /**
+   * @return the upgrades folder, or {@code null} if not set
+   */
+  public String getUpgradesFolder() {
+    return upgradesFolder;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/main/java/org/apache/ambari/server/state/stack/UpgradePack.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/UpgradePack.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/UpgradePack.java
new file mode 100644
index 0000000..3057db3
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/UpgradePack.java
@@ -0,0 +1,159 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+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.XmlElementWrapper;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlTransient;
+
+import org.apache.ambari.server.state.stack.upgrade.Batch;
+import org.apache.ambari.server.state.stack.upgrade.Task;
+
+/**
+ * Represents an upgrade pack.
+ */
+@XmlRootElement(name="upgrade")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class UpgradePack {
+
+  @XmlElement(name="target")
+  private String target;
+
+
+  @XmlElementWrapper(name="order")
+  @XmlElement(name="service")
+  private List<OrderService> services;
+
+  @XmlElementWrapper(name="processing")
+  @XmlElement(name="service")
+  private List<ProcessingService> processing;
+
+  @XmlTransient
+  private Map<String, List<String>> m_orders = null;
+  @XmlTransient
+  private Map<String, Map<String, ProcessingComponent>> m_process = null;
+
+  /**
+   * @return the target version for the upgrade pack
+   */
+  public String getTarget() {
+    return target;
+  }
+
+  /**
+   * Gets the order by which services and components should be upgraded.
+   * @return a map of service_name -> list of component_name.
+   */
+  public Map<String, List<String>> getOrder() {
+
+    if (null == m_orders) {
+      m_orders = new LinkedHashMap<String, List<String>>();
+
+      for (OrderService order : services) {
+        m_orders.put(order.name, order.components);
+      }
+    }
+
+    return m_orders;
+  }
+
+  /**
+   * Gets the tasks by which services and components should be upgraded.
+   * @return a map of service_name -> map(component_name -> process).
+   */
+  public Map<String, Map<String, ProcessingComponent>> getTasks() {
+
+    if (null == m_process) {
+      m_process = new LinkedHashMap<String, Map<String, ProcessingComponent>>();
+
+      for (ProcessingService svc : processing) {
+        if (!m_process.containsKey(svc.name)) {
+          m_process.put(svc.name, new LinkedHashMap<String, ProcessingComponent>());
+        }
+
+        Map<String, ProcessingComponent> componentMap = m_process.get(svc.name);
+
+        for (ProcessingComponent pc : svc.components) {
+          componentMap.put(pc.name, pc);
+        }
+      }
+    }
+
+    return m_process;
+  }
+
+
+  /**
+   * A service definition that holds a list of componenents in the 'order' element.
+   */
+  public static class OrderService {
+
+    @XmlAttribute
+    public String name;
+
+    @XmlElement(name="component")
+    public List<String> components;
+  }
+
+  /**
+   * A service definition in the 'processing' element.
+   */
+  public static class ProcessingService {
+
+    @XmlAttribute
+    public String name;
+
+    @XmlElement(name="component")
+    public List<ProcessingComponent> components;
+  }
+
+  /**
+   * A component definition in the 'processing/service' path.
+   */
+  public static class ProcessingComponent {
+
+    @XmlAttribute
+    public String name;
+
+    @XmlElementWrapper(name="pre-upgrade")
+    @XmlElement(name="task")
+    public List<Task> preTasks;
+
+    @XmlElementWrapper(name="upgrade")
+    @XmlElement(name="task")
+    public List<Task> tasks;
+
+    @XmlElementWrapper(name="post-upgrade")
+    @XmlElement(name="task")
+    public List<Task> postTasks;
+
+    @XmlElement(name="batch")
+    public Batch batch;
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Batch.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Batch.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Batch.java
new file mode 100644
index 0000000..cbd2e79
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Batch.java
@@ -0,0 +1,51 @@
+/**
+ * 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 javax.xml.bind.annotation.XmlSeeAlso;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * Base class to identify how a component should be upgraded (optional)
+ */
+@XmlSeeAlso(value={CountBatch.class, PercentBatch.class, ConditionalBatch.class})
+public abstract class Batch {
+
+  /**
+   * @return the batch type
+   */
+  public abstract Type getType();
+
+  /**
+   * Identifies the type of batch
+   */
+  public enum Type {
+    /**
+     * Batch by <i>n</i> instance at a time
+     */
+    COUNT,
+    /**
+     * Batch by <i>x</i>% at a time
+     */
+    PERCENT,
+    /**
+     * Batch by an inital <i>x</i>%, then after confirmation batch <i>y</i>% at a time.
+     */
+    CONDITIONAL
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConditionalBatch.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConditionalBatch.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConditionalBatch.java
new file mode 100644
index 0000000..e8e2c07
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConditionalBatch.java
@@ -0,0 +1,46 @@
+/**
+ * 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 javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * Upgrade batch that should happen by percentage.  After the inital percentage,
+ * the remaining nodes are upgraded incrementally.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name="conditional")
+public class ConditionalBatch extends Batch {
+
+  @XmlElement(name="initial")
+  public int initial = 0;
+
+  @XmlElement(name="remaining")
+  public int remaining = 0;
+
+  @Override
+  public Type getType() {
+    return Batch.Type.CONDITIONAL;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/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
new file mode 100644
index 0000000..a4b2fc5
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ConfigureTask.java
@@ -0,0 +1,47 @@
+/**
+ * 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 javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * Upgrade task that represents a configuration should change.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name="configure")
+public class ConfigureTask extends Task {
+
+  @XmlElement(name="type")
+  public String config;
+
+  @XmlElement(name="key")
+  public String key;
+
+  @XmlElement(name="value")
+  public String value;
+
+  @Override
+  public Task.Type getType() {
+    return Task.Type.CONFIGURE;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/CountBatch.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/CountBatch.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/CountBatch.java
new file mode 100644
index 0000000..b95b616
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/CountBatch.java
@@ -0,0 +1,42 @@
+/**
+ * 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 javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * Upgrade batch that should happen one at a time.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name="count")
+public class CountBatch extends Batch {
+
+  @XmlElement(name="count")
+  public int count = 0;
+
+  @Override
+  public Type getType() {
+    return Batch.Type.COUNT;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ExecuteTask.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ExecuteTask.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ExecuteTask.java
new file mode 100644
index 0000000..2677d7c
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ExecuteTask.java
@@ -0,0 +1,41 @@
+/**
+ * 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 javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * Used to represent an execution that should occur on an agent.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name="execute")
+public class ExecuteTask extends Task {
+
+  @XmlElement(name="command")
+  public String command;
+
+  @Override
+  public Task.Type getType() {
+    return Task.Type.EXECUTE;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ManualTask.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ManualTask.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ManualTask.java
new file mode 100644
index 0000000..ee92394
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ManualTask.java
@@ -0,0 +1,42 @@
+/**
+ * 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 javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * Identifies that an upgrade step that requires confirmation before continuing.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name="manual")
+public class ManualTask extends Task {
+
+  @XmlElement(name="message")
+  public String message;
+
+  @Override
+  public Task.Type getType() {
+    return Task.Type.MANUAL;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/PercentBatch.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/PercentBatch.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/PercentBatch.java
new file mode 100644
index 0000000..2c3604e
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/PercentBatch.java
@@ -0,0 +1,43 @@
+/**
+ * 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 javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * Upgrade batch that should happen by percentage.  After each percentage
+ * completes, continue to the next step.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name="percent")
+public class PercentBatch extends Batch {
+
+  @XmlElement(name="percent")
+  public int count = 0;
+
+  @Override
+  public Type getType() {
+    return Batch.Type.PERCENT;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Task.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Task.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Task.java
new file mode 100644
index 0000000..3ebde36
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/Task.java
@@ -0,0 +1,54 @@
+/**
+ * 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 javax.xml.bind.annotation.XmlSeeAlso;
+import javax.xml.bind.annotation.XmlType;
+
+
+/**
+ * Base class to identify the items that could possibly occur during an upgrade
+ */
+@XmlSeeAlso(value={ExecuteTask.class, ConfigureTask.class, ManualTask.class})
+@XmlType
+public abstract class Task {
+
+  /**
+   * @return the type of the task
+   */
+  public abstract Type getType();
+
+
+  /**
+   * Identifies the type of task.
+   */
+  public enum Type {
+    /**
+     * Task that is executed on a host.
+     */
+    EXECUTE,
+    /**
+     * Task that alters a configuration.
+     */
+    CONFIGURE,
+    /**
+     * Task that displays a message and must be confirmed before continuing
+     */
+    MANUAL
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/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
new file mode 100644
index 0000000..ec402c0
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml
@@ -0,0 +1,54 @@
+<?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 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <order>
+    <service name="ZOOKEEPER">
+      <component>ZOOKEEPER_SERVER</component>
+      <component>ZOOKEEPER_CLIENT</component>
+    </service>
+    <service name="HDFS">
+      <component>JOURNALNODE</component>
+      <component>NAMENODE</component>
+      <component>DATANODE</component>
+    </service>
+  </order>
+  <processing>
+    <service name="HDFS">
+      <component name="NAMENODE">
+        <upgrade>
+          <task xsi:type="execute">
+            <command>su - {hdfs-user} -c 'hdp-select hadoop-hdfs-namenode {version}'</command>
+          </task>
+        </upgrade>
+      </component>
+    </service>
+    <service name="HDFS">
+      <component name="DATANODE">
+        <batch xsi:type="conditional">
+          <initial>10</initial>
+          <remaining>25</remaining>
+        </batch>
+        <upgrade>
+          <task xsi:type="execute">
+            <command>su - {hdfs-user} -c 'hdp-select hadoop-hdfs-datanode {version}'</command>
+          </task>
+        </upgrade>
+      </component>
+    </service>
+  </processing>
+</upgrade>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/test/java/org/apache/ambari/server/state/stack/UpgradePackTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/stack/UpgradePackTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/stack/UpgradePackTest.java
new file mode 100644
index 0000000..551cac3
--- /dev/null
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/stack/UpgradePackTest.java
@@ -0,0 +1,198 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.ambari.server.api.services.AmbariMetaInfo;
+import org.apache.ambari.server.orm.GuiceJpaInitializer;
+import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
+import org.apache.ambari.server.state.stack.UpgradePack.ProcessingComponent;
+import org.apache.ambari.server.state.stack.upgrade.Batch;
+import org.apache.ambari.server.state.stack.upgrade.ConditionalBatch;
+import org.apache.ambari.server.state.stack.upgrade.ConfigureTask;
+import org.apache.ambari.server.state.stack.upgrade.CountBatch;
+import org.apache.ambari.server.state.stack.upgrade.ExecuteTask;
+import org.apache.ambari.server.state.stack.upgrade.ManualTask;
+import org.apache.ambari.server.state.stack.upgrade.Task;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.persist.PersistService;
+
+/**
+ * Tests for the upgrade pack
+ */
+public class UpgradePackTest {
+
+  private Injector injector;
+  private AmbariMetaInfo ambariMetaInfo;
+
+  @Before
+  public void before() throws Exception {
+    injector = Guice.createInjector(new InMemoryDefaultTestModule());
+    injector.getInstance(GuiceJpaInitializer.class);
+
+    ambariMetaInfo = injector.getInstance(AmbariMetaInfo.class);
+    ambariMetaInfo.init();
+  }
+
+  @After
+  public void teardown() {
+    injector.getInstance(PersistService.class).stop();
+  }
+
+  @Test
+  public void testExistence() throws Exception {
+    Map<String, UpgradePack> upgrades = ambariMetaInfo.getUpgradePacks("foo", "bar");
+    assertTrue(upgrades.isEmpty());
+
+    upgrades = ambariMetaInfo.getUpgradePacks("HDP", "2.1.1");
+    assertTrue(upgrades.size() > 0);
+    assertTrue(upgrades.containsKey("upgrade_test"));
+  }
+
+  @Test
+  public void testUpgradeParsing() throws Exception {
+    Map<String, UpgradePack> upgrades = ambariMetaInfo.getUpgradePacks("HDP", "2.1.1");
+    assertTrue(upgrades.size() > 0);
+
+    Map<String, List<String>> expectedOrder = new LinkedHashMap<String, List<String>>() {{
+      put("ZOOKEEPER", Arrays.asList("ZOOKEEPER_SERVER", "ZOOKEEPER_CLIENT"));
+      put("HDFS", Arrays.asList("JOURNALNODE", "NAMENODE", "DATANODE"));
+    }};
+
+
+    UpgradePack up = upgrades.values().iterator().next();
+    assertEquals("2.2.*", up.getTarget());
+
+    // !!! test the orders
+    assertEquals(expectedOrder.size(), up.getOrder().size());
+
+    int i = 0;
+    for (Entry<String, List<String>> entry : expectedOrder.entrySet()) {
+      assertTrue(up.getOrder().containsKey(entry.getKey()));
+      assertEquals(i++, indexOf(up.getOrder(), entry.getKey()));
+
+      int j = 0;
+      for (String comp : entry.getValue()) {
+        assertEquals(comp, up.getOrder().get(entry.getKey()).get(j++));
+      }
+    }
+
+    Map<String, List<String>> expectedStages = new LinkedHashMap<String, List<String>>() {{
+      put("ZOOKEEPER", Arrays.asList("ZOOKEEPER_SERVER"));
+      put("HDFS", Arrays.asList("NAMENODE", "DATANODE"));
+    }};
+
+    // !!! test the tasks
+    i = 0;
+    for (Entry<String, List<String>> entry : expectedStages.entrySet()) {
+      assertTrue(up.getTasks().containsKey(entry.getKey()));
+      assertEquals(i++, indexOf(up.getTasks(), entry.getKey()));
+
+      // check that the number of components matches
+      assertEquals(entry.getValue().size(), up.getTasks().get(entry.getKey()).size());
+
+      // check component ordering
+      int j = 0;
+      for (String comp : entry.getValue()) {
+        assertEquals(j++, indexOf(up.getTasks().get(entry.getKey()), comp));
+      }
+    }
+
+    // !!! test specific tasks
+    assertTrue(up.getTasks().containsKey("HDFS"));
+    assertTrue(up.getTasks().get("HDFS").containsKey("NAMENODE"));
+
+    ProcessingComponent pc = up.getTasks().get("HDFS").get("NAMENODE");
+    assertNull(pc.batch);
+    assertNull(pc.preTasks);
+    assertNull(pc.postTasks);
+    assertNotNull(pc.tasks);
+    assertEquals(3, pc.tasks.size());
+
+    assertEquals(Task.Type.EXECUTE, pc.tasks.get(0).getType());
+    assertEquals(ExecuteTask.class, pc.tasks.get(0).getClass());
+    assertEquals("su - {hdfs-user} -c 'dosomething'",
+        ExecuteTask.class.cast(pc.tasks.get(0)).command);
+
+    assertEquals(Task.Type.CONFIGURE, pc.tasks.get(1).getType());
+    assertEquals(ConfigureTask.class, pc.tasks.get(1).getClass());
+    assertEquals("hdfs-site",
+        ConfigureTask.class.cast(pc.tasks.get(1)).config);
+    assertEquals("myproperty",
+        ConfigureTask.class.cast(pc.tasks.get(1)).key);
+    assertEquals("mynewvalue",
+        ConfigureTask.class.cast(pc.tasks.get(1)).value);
+
+    assertEquals(Task.Type.MANUAL, pc.tasks.get(2).getType());
+    assertEquals(ManualTask.class, pc.tasks.get(2).getClass());
+    assertEquals("Update your database",
+        ManualTask.class.cast(pc.tasks.get(2)).message);
+
+    assertTrue(up.getTasks().containsKey("ZOOKEEPER"));
+    assertTrue(up.getTasks().get("ZOOKEEPER").containsKey("ZOOKEEPER_SERVER"));
+
+    pc = up.getTasks().get("HDFS").get("DATANODE");
+    assertNotNull(pc.batch);
+    assertEquals(Batch.Type.CONDITIONAL, pc.batch.getType());
+    assertEquals(15, ConditionalBatch.class.cast(pc.batch).initial);
+    assertEquals(50, ConditionalBatch.class.cast(pc.batch).remaining);
+
+    pc = up.getTasks().get("ZOOKEEPER").get("ZOOKEEPER_SERVER");
+    assertNotNull(pc.preTasks);
+    assertEquals(1, pc.preTasks.size());
+    assertNotNull(pc.postTasks);
+    assertEquals(1, pc.postTasks.size());
+    assertNotNull(pc.tasks);
+    assertEquals(1, pc.tasks.size());
+    assertNotNull(pc.batch);
+    assertEquals(Batch.Type.COUNT, pc.batch.getType());
+    assertEquals(2, CountBatch.class.cast(pc.batch).count);
+
+  }
+
+
+  private int indexOf(Map<String, ?> map, String keyToFind) {
+    int result = -1;
+
+    int i = 0;
+    for (Entry<String, ?> entry : map.entrySet()) {
+      if (entry.getKey().equals(keyToFind))
+        return i;
+      i++;
+    }
+
+    return result;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/b17637c9/ambari-server/src/test/resources/stacks/HDP/2.1.1/upgrades/upgrade_test.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/resources/stacks/HDP/2.1.1/upgrades/upgrade_test.xml b/ambari-server/src/test/resources/stacks/HDP/2.1.1/upgrades/upgrade_test.xml
new file mode 100644
index 0000000..90e907b
--- /dev/null
+++ b/ambari-server/src/test/resources/stacks/HDP/2.1.1/upgrades/upgrade_test.xml
@@ -0,0 +1,83 @@
+<?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 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <target>2.2.*</target>
+  <order>
+    <service name="ZOOKEEPER">
+      <component>ZOOKEEPER_SERVER</component>
+      <component>ZOOKEEPER_CLIENT</component>
+    </service>
+    <service name="HDFS">
+      <component>JOURNALNODE</component>
+      <component>NAMENODE</component>
+      <component>DATANODE</component>
+    </service>
+  </order>
+  <processing>
+    <service name="ZOOKEEPER">
+      <component name="ZOOKEEPER_SERVER">
+        <batch xsi:type="count">
+          <count>2</count>
+        </batch>
+        <pre-upgrade>
+          <task xsi:type="manual">
+            <message>this is pre</message>
+          </task>
+        </pre-upgrade>
+        <upgrade>
+          <task xsi:type="execute">
+            <command>ls -l</command>
+          </task>
+        </upgrade>
+        <post-upgrade>
+          <task xsi:type="manual">
+            <message>this is post</message>
+          </task>
+        </post-upgrade>
+      </component>
+    </service>
+    <service name="HDFS">
+      <component name="NAMENODE">
+        <upgrade>
+          <task xsi:type="execute">
+            <command>su - {hdfs-user} -c 'dosomething'</command>
+          </task>
+          <task xsi:type="configure">
+            <type>hdfs-site</type>
+            <key>myproperty</key>
+            <value>mynewvalue</value>
+          </task>
+          <task xsi:type="manual">
+            <message>Update your database</message>
+          </task>
+        </upgrade>
+      </component>
+      <component name="DATANODE">
+        <batch xsi:type="conditional">
+          <initial>15</initial>
+          <remaining>50</remaining>
+        </batch>
+        <upgrade>
+          <task xsi:type="execute">
+            <command>ls</command>
+          </task>
+        </upgrade>
+      </component>
+    </service>
+  </processing>
+</upgrade>