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/12/11 11:21:55 UTC

ambari git commit: AMBARI-8574. Upgrade Execute: add pre- and post-cluster execute tasks (ncole)

Repository: ambari
Updated Branches:
  refs/heads/trunk a51afbe54 -> e9d39f951


AMBARI-8574. Upgrade Execute: add pre- and post-cluster execute tasks (ncole)


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

Branch: refs/heads/trunk
Commit: e9d39f9513efb5b07556c313c651c1e5f4d458a6
Parents: a51afbe
Author: Nate Cole <nc...@hortonworks.com>
Authored: Wed Dec 10 17:04:01 2014 -0500
Committer: Nate Cole <nc...@hortonworks.com>
Committed: Thu Dec 11 04:58:35 2014 -0500

----------------------------------------------------------------------
 .../ambari/server/actionmanager/Stage.java      |  64 ++++++---
 .../internal/UpgradeResourceProvider.java       |  55 +++++++-
 .../upgrades/ManualStageAction.java             |  40 ++++++
 .../ambari/server/state/UpgradeHelper.java      |  98 +++++++++-----
 .../state/stack/upgrade/ClusterGrouping.java    | 134 +++++++++++++++++++
 .../server/state/stack/upgrade/Grouping.java    |   4 +-
 .../stacks/HDP/2.2/upgrades/upgrade-2.2.xml     |  21 ++-
 .../internal/UpgradeResourceProviderTest.java   |  21 ++-
 .../ambari/server/state/UpgradeHelperTest.java  |  20 +--
 .../stacks/HDP/2.1.1/upgrades/upgrade_test.xml  |  35 +++++
 10 files changed, 416 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/e9d39f95/ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Stage.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Stage.java b/ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Stage.java
index 974acd4..28d9b8d 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Stage.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Stage.java
@@ -17,7 +17,6 @@
  */
 package org.apache.ambari.server.actionmanager;
 
-import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -106,7 +105,7 @@ public class Stage {
     requestId = stageEntity.getRequestId();
     stageId = stageEntity.getStageId();
     logDir = stageEntity.getLogInfo();
-    
+
     long clusterId = stageEntity.getClusterId().longValue();
     if (-1L != clusterId) {
       try {
@@ -116,7 +115,7 @@ public class Stage {
             Long.valueOf(clusterId), Long.valueOf(stageId));
       }
     }
-    
+
     requestContext = stageEntity.getRequestContext();
     clusterHostInfo = stageEntity.getClusterHostInfo();
     commandParamsStage = stageEntity.getCommandParamsStage();
@@ -208,7 +207,7 @@ public class Stage {
   public void setClusterHostInfo(String clusterHostInfo) {
     this.clusterHostInfo = clusterHostInfo;
   }
- 
+
   public String getCommandParamsStage() {
     return commandParamsStage;
   }
@@ -247,7 +246,7 @@ public class Stage {
   public String getActionId() {
     return StageUtils.getActionId(requestId, getStageId());
   }
-  
+
   private synchronized ExecutionCommandWrapper addGenericExecutionCommand(String clusterName, String hostName, Role role, RoleCommand command, ServiceComponentHostEvent event){
     //used on stage creation only, no need to check if wrappers loaded
     HostRoleCommand hrc = new HostRoleCommand(hostName, role, event, command);
@@ -259,9 +258,9 @@ public class Stage {
     cmd.setCommandId(this.getActionId());
     cmd.setRole(role.name());
     cmd.setRoleCommand(command);
-    
+
     cmd.setServiceName("");
-    
+
     Map<String, HostRoleCommand> hrcMap = this.hostRoleCommands.get(hostName);
     if (hrcMap == null) {
       hrcMap = new LinkedHashMap<String, HostRoleCommand>();
@@ -278,7 +277,7 @@ public class Stage {
       execCmdList = new ArrayList<ExecutionCommandWrapper>();
       this.commandsToSend.put(hostName, execCmdList);
     }
-    
+
     if (execCmdList.contains(wrapper)) {
       //todo: proper exception
       throw new RuntimeException(
@@ -297,7 +296,7 @@ public class Stage {
   public synchronized void addHostRoleExecutionCommand(String host, Role role,  RoleCommand command,
       ServiceComponentHostEvent event, String clusterName, String serviceName) {
     ExecutionCommandWrapper commandWrapper = addGenericExecutionCommand(clusterName, host, role, command, event);
-    
+
     commandWrapper.getExecutionCommand().setServiceName(serviceName);
   }
 
@@ -320,10 +319,41 @@ public class Stage {
    *                      timeout will be used
    */
   public synchronized void addServerActionCommand(String actionName, Role role, RoleCommand command,
-                                                  String clusterName, ServiceComponentHostServerActionEvent event,
+      String clusterName, ServiceComponentHostServerActionEvent event,
+      @Nullable Map<String, String> commandParams,
+      @Nullable Integer timeout) {
+
+    addServerActionCommand(actionName, role, command,
+        clusterName, StageUtils.getHostName(), event, commandParams, timeout);
+  }
+
+  /**
+   * THIS METHOD IS TO WORKAROUND A BUG!  The assumption of the framework
+   * is that the Ambari Server is installed on a host WITHIN the cluster, which
+   * is not always true.  This method adds a host parameter.
+   *
+   * Creates server-side execution command.
+   * <p/>
+   * The action name for this command is expected to be the classname of a
+   * {@link org.apache.ambari.server.serveraction.ServerAction} implementation which will be
+   * instantiated and invoked as needed.
+   *
+   * @param actionName    a String declaring the action name (in the form of a classname) to execute
+   * @param role          the Role for this command
+   * @param command       the RoleCommand for this command
+   * @param clusterName   a String identifying the cluster on which to to execute this command
+   * @param host          the name of the host
+   * @param event         a ServiceComponentHostServerActionEvent
+   * @param commandParams a Map of String to String data used to pass to the action - this may be
+   *                      empty or null if no data is relevant
+   * @param timeout       an Integer declaring the timeout for this action - if null, a default
+   *                      timeout will be used
+   */
+  public synchronized void addServerActionCommand(String actionName, Role role, RoleCommand command,
+                                                  String clusterName, String hostName, ServiceComponentHostServerActionEvent event,
                                                   @Nullable Map<String, String> commandParams,
                                                   @Nullable Integer timeout) {
-    ExecutionCommandWrapper commandWrapper = addGenericExecutionCommand(clusterName, StageUtils.getHostName(), role, command, event);
+    ExecutionCommandWrapper commandWrapper = addGenericExecutionCommand(clusterName, hostName, role, command, event);
     ExecutionCommand cmd = commandWrapper.getExecutionCommand();
 
     Map<String, String> cmdParams = new HashMap<String, String>();
@@ -347,15 +377,15 @@ public class Stage {
     ExecutionCommandWrapper commandWrapper = addGenericExecutionCommand(clusterName, hostName, Role.AMBARI_SERVER_ACTION, RoleCommand.ABORT, null);
     ExecutionCommand cmd = commandWrapper.getExecutionCommand();
     cmd.setCommandType(AgentCommandType.CANCEL_COMMAND);
-    
+
     Assert.notEmpty(cancelTargets, "Provided targets task Id are empty.");
-    
+
     Map<String, String> roleParams = new HashMap<String, String>();
-    
+
     roleParams.put("cancelTaskIdTargets", StringUtils.join(cancelTargets, ','));
     cmd.setRoleParams(roleParams);
   }
-  
+
   /**
    *
    * @return list of hosts
@@ -397,11 +427,11 @@ public class Stage {
   public String getClusterName() {
     return clusterName;
   }
-  
+
   public long getClusterId() {
     return clusterId;
   }
-  
+
 
   public String getRequestContext() {
     return requestContext;

http://git-wip-us.apache.org/repos/asf/ambari/blob/e9d39f95/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 9cce147..3bc5c4e 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
@@ -28,11 +28,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.Role;
+import org.apache.ambari.server.RoleCommand;
 import org.apache.ambari.server.StaticallyInject;
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.actionmanager.HostRoleCommand;
@@ -60,6 +61,7 @@ import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
 import org.apache.ambari.server.orm.entities.UpgradeEntity;
 import org.apache.ambari.server.orm.entities.UpgradeGroupEntity;
 import org.apache.ambari.server.orm.entities.UpgradeItemEntity;
+import org.apache.ambari.server.serveraction.upgrades.ManualStageAction;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.ConfigHelper;
 import org.apache.ambari.server.state.StackId;
@@ -68,6 +70,7 @@ import org.apache.ambari.server.state.UpgradeHelper.UpgradeGroupHolder;
 import org.apache.ambari.server.state.stack.UpgradePack;
 import org.apache.ambari.server.state.stack.upgrade.StageWrapper;
 import org.apache.ambari.server.state.stack.upgrade.TaskWrapper;
+import org.apache.ambari.server.state.svccomphost.ServiceComponentHostServerActionEvent;
 import org.apache.ambari.server.utils.StageUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -430,6 +433,9 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
       case SERVICE_CHECK:
         makeServiceCheckStage(cluster, request, version, entity, wrapper);
         break;
+      case MANUAL:
+        makeManualStage(cluster, request, version, entity, wrapper);
+        break;
     }
 
   }
@@ -576,4 +582,51 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
     request.addStages(Collections.singletonList(stage));
   }
 
+  private void makeManualStage(Cluster cluster, RequestStageContainer request, String version,
+      UpgradeItemEntity entity, StageWrapper wrapper) throws AmbariException {
+
+    Map<String, String> restartCommandParams = new HashMap<String, String>();
+    restartCommandParams.put("version", version);
+
+    ActionExecutionContext actionContext = new ActionExecutionContext(
+        cluster.getClusterName(), Role.AMBARI_SERVER_ACTION.toString(),
+        Collections.<RequestResourceFilter>emptyList(),
+        restartCommandParams);
+    actionContext.setTimeout(Short.valueOf((short)-1));
+
+    ExecuteCommandJson jsons = commandExecutionHelper.get().getCommandJson(
+        actionContext, cluster);
+
+
+    Stage stage = stageFactory.get().createNew(request.getId().longValue(),
+        "/tmp/ambari",
+        cluster.getClusterName(),
+        cluster.getClusterId(),
+        entity.getText(),
+        jsons.getClusterHostInfo(),
+        jsons.getCommandParamsForStage(),
+        jsons.getHostParamsForStage());
+
+    long stageId = request.getLastStageId() + 1;
+    if (0L == stageId) {
+      stageId = 1L;
+    }
+    stage.setStageId(stageId);
+    entity.setStageId(Long.valueOf(stageId));
+
+    // !!! hack hack hack
+    String host = cluster.getAllHostsDesiredConfigs().keySet().iterator().next();
+
+    stage.addServerActionCommand(ManualStageAction.class.getName(),
+        Role.AMBARI_SERVER_ACTION,
+        RoleCommand.EXECUTE,
+        cluster.getClusterName(), host,
+        new ServiceComponentHostServerActionEvent(StageUtils.getHostName(), System.currentTimeMillis()),
+        Collections.<String, String>emptyMap(), 1200);
+
+    request.addStages(Collections.singletonList(stage));
+
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/e9d39f95/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ManualStageAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ManualStageAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ManualStageAction.java
new file mode 100644
index 0000000..4460dbe
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ManualStageAction.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ambari.server.serveraction.upgrades;
+
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.actionmanager.HostRoleStatus;
+import org.apache.ambari.server.agent.CommandReport;
+import org.apache.ambari.server.serveraction.AbstractServerAction;
+
+/**
+ * Action that represents a manual stage.
+ */
+public class ManualStageAction extends AbstractServerAction {
+
+  @Override
+  public CommandReport execute(
+      ConcurrentMap<String, Object> requestSharedDataContext)
+      throws AmbariException, InterruptedException {
+    // TODO Auto-generated method stub
+    return createCommandReport(0, HostRoleStatus.COMPLETED, "{}", "", "");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/e9d39f95/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 8201a8b..7bd7591 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
@@ -23,13 +23,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.ambari.server.controller.internal.RequestImpl;
 import org.apache.ambari.server.controller.internal.RequestResourceProvider;
 import org.apache.ambari.server.controller.internal.StageResourceProvider;
-import org.apache.ambari.server.controller.internal.UpgradeResourceProvider;
 import org.apache.ambari.server.controller.predicate.AndPredicate;
-import org.apache.ambari.server.controller.predicate.EqualsPredicate;
-import org.apache.ambari.server.controller.predicate.OrPredicate;
 import org.apache.ambari.server.controller.spi.ClusterController;
 import org.apache.ambari.server.controller.spi.NoSuchParentResourceException;
 import org.apache.ambari.server.controller.spi.NoSuchResourceException;
@@ -44,11 +40,11 @@ import org.apache.ambari.server.controller.utilities.PredicateBuilder;
 import org.apache.ambari.server.controller.utilities.PropertyHelper;
 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.ClusterGrouping;
 import org.apache.ambari.server.state.stack.upgrade.Grouping;
 import org.apache.ambari.server.state.stack.upgrade.StageWrapper;
 import org.apache.ambari.server.state.stack.upgrade.StageWrapperBuilder;
 import org.apache.ambari.server.state.stack.upgrade.TaskWrapper;
-import org.codehaus.jackson.map.ser.PropertyBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,6 +68,14 @@ public class UpgradeHelper {
     List<UpgradeGroupHolder> groups = new ArrayList<UpgradeGroupHolder>();
 
     for (Grouping group : upgradePack.getGroups()) {
+      if (ClusterGrouping.class.isInstance(group)) {
+        UpgradeGroupHolder groupHolder = getClusterGroupHolder(cluster, (ClusterGrouping) group);
+        if (null != groupHolder) {
+          groups.add(groupHolder);
+          continue;
+        }
+      }
+
       UpgradeGroupHolder groupHolder = new UpgradeGroupHolder();
       groupHolder.name = group.name;
       groupHolder.title = group.title;
@@ -91,7 +95,6 @@ public class UpgradeHelper {
           }
 
           Set<String> componentHosts = getClusterHosts(cluster, service.serviceName, component);
-
           if (0 == componentHosts.size()) {
             continue;
           }
@@ -104,11 +107,15 @@ public class UpgradeHelper {
 
       List<StageWrapper> proxies = builder.build();
 
-      if (LOG.isDebugEnabled()) {
+      groupHolder.items = proxies;
+    }
+
+    if (LOG.isDebugEnabled()) {
+      for (UpgradeGroupHolder group : groups) {
         LOG.debug(group.name);
 
         int i = 0;
-        for (StageWrapper proxy : proxies) {
+        for (StageWrapper proxy : group.items) {
           LOG.debug("  Stage {}", Integer.valueOf(i++));
           int j = 0;
 
@@ -117,37 +124,11 @@ public class UpgradeHelper {
           }
         }
       }
-
-      groupHolder.items = proxies;
     }
 
     return groups;
-
   }
 
-  /**
-   * @param cluster the cluster
-   * @param serviceName name of the service
-   * @param componentName name of the component
-   * @return the set of hosts for the provided service and component
-   */
-  private Set<String> getClusterHosts(Cluster cluster, String serviceName, String componentName) {
-    Map<String, Service> services = cluster.getServices();
-
-    if (!services.containsKey(serviceName)) {
-      return Collections.emptySet();
-    }
-
-    Service service = services.get(serviceName);
-    Map<String, ServiceComponent> components = service.getServiceComponents();
-
-    if (!components.containsKey(componentName) ||
-        components.get(componentName).getServiceComponentHosts().size() == 0) {
-      return Collections.emptySet();
-    }
-
-    return components.get(componentName).getServiceComponentHosts().keySet();
-  }
 
   /**
    * Short-lived objects that hold information about upgrade groups
@@ -251,4 +232,53 @@ public class UpgradeHelper {
     return resources.iterator().next();
   }
 
+  /**
+   * @param cluster the cluster
+   * @param serviceName name of the service
+   * @param componentName name of the component
+   * @return the set of hosts for the provided service and component
+   */
+  public Set<String> getClusterHosts(Cluster cluster, String serviceName, String componentName) {
+    Map<String, Service> services = cluster.getServices();
+
+    if (!services.containsKey(serviceName)) {
+      return Collections.emptySet();
+    }
+
+    Service service = services.get(serviceName);
+    Map<String, ServiceComponent> components = service.getServiceComponents();
+
+    if (!components.containsKey(componentName) ||
+        components.get(componentName).getServiceComponentHosts().size() == 0) {
+      return Collections.emptySet();
+    }
+
+    return components.get(componentName).getServiceComponentHosts().keySet();
+  }
+
+  /**
+   * Special handling for ClusterGrouping.
+   * @param cluster the cluster
+   * @param grouping the grouping
+   * @return the holder, or {@code null} if there are no clustergrouping tasks.
+   */
+  private UpgradeGroupHolder getClusterGroupHolder(Cluster cluster, ClusterGrouping grouping) {
+
+    grouping.getBuilder().setHelpers(this, cluster);
+    List<StageWrapper> wrappers = grouping.getBuilder().build();
+
+    if (wrappers.size() > 0) {
+      UpgradeGroupHolder holder = new UpgradeGroupHolder();
+      holder.name = grouping.name;
+      holder.title = grouping.title;
+      holder.items = wrappers;
+
+      return holder;
+    }
+
+
+    return null;
+
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/e9d39f95/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ClusterGrouping.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ClusterGrouping.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ClusterGrouping.java
new file mode 100644
index 0000000..984e270
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/ClusterGrouping.java
@@ -0,0 +1,134 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+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.Cluster;
+import org.apache.ambari.server.state.UpgradeHelper;
+import org.apache.ambari.server.state.stack.UpgradePack;
+import org.apache.ambari.server.state.stack.UpgradePack.ProcessingComponent;
+
+/**
+ * Used to represent cluster-based operations.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name="cluster")
+public class ClusterGrouping extends Grouping {
+
+
+  @XmlElement(name="execute-stage")
+  private List<ExecuteStage> executionStages;
+
+  @XmlTransient
+  private ClusterBuilder m_builder = new ClusterBuilder();
+
+  @Override
+  public ClusterBuilder getBuilder() {
+    return m_builder;
+  }
+
+
+  private static class ExecuteStage {
+    @XmlAttribute(name="title")
+    public String title;
+
+    @XmlAttribute(name="service")
+    public String service;
+
+    @XmlAttribute(name="component")
+    public String component;
+
+    @XmlElement(name="task")
+    public Task task;
+  }
+
+  public class ClusterBuilder extends StageWrapperBuilder {
+    private UpgradeHelper m_helper = null;
+    private Cluster m_cluster = null;
+
+    /**
+     * @param cluster the cluster to use with this builder
+     */
+    public void setHelpers(UpgradeHelper helper, Cluster cluster) {
+      m_helper = helper;
+      m_cluster = cluster;
+    }
+
+    @Override
+    public void add(Set<String> hosts, String service, ProcessingComponent pc) {
+      // !!! no-op in this case
+    }
+
+    @Override
+    public List<StageWrapper> build() {
+      if (null == ClusterGrouping.this.executionStages) {
+        return Collections.emptyList();
+      }
+
+      List<StageWrapper> results = new ArrayList<StageWrapper>();
+
+      for (ExecuteStage execution : executionStages) {
+        Task task = execution.task;
+
+        StageWrapper wrapper = null;
+
+        if (null != execution.service && null != execution.component) {
+          Set<String> hosts = m_helper.getClusterHosts(m_cluster, execution.service, execution.component);
+          // !!! FIXME other types
+          if (hosts.size() > 0 && task.getType() == Task.Type.EXECUTE) {
+            wrapper = new StageWrapper(
+                StageWrapper.Type.RU_TASKS,
+                execution.title,
+                new TaskWrapper(execution.service, execution.component, hosts, task));
+          }
+
+        } else {
+          switch (task.getType()) {
+            case MANUAL:
+              wrapper = new StageWrapper(
+                  StageWrapper.Type.MANUAL,
+                  execution.title,
+                  new TaskWrapper(null, null, Collections.<String>emptySet(), task));
+              break;
+            default:
+              break;
+          }
+        }
+
+        if (null != wrapper) {
+          results.add(wrapper);
+        }
+      }
+
+      return results;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/e9d39f95/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 c45f005..15399b5 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
@@ -34,7 +34,7 @@ import org.apache.commons.lang.StringUtils;
 /**
  *
  */
-@XmlSeeAlso(value = { ColocatedGrouping.class })
+@XmlSeeAlso(value = { ColocatedGrouping.class, ClusterGrouping.class })
 public class Grouping {
 
   @XmlAttribute(name="name")
@@ -44,7 +44,7 @@ public class Grouping {
   public String title;
 
   @XmlElement(name="service")
-  public List<UpgradePack.OrderService> services;
+  public List<UpgradePack.OrderService> services = new ArrayList<UpgradePack.OrderService>();
 
   /**
    * Gets the default builder.

http://git-wip-us.apache.org/repos/asf/ambari/blob/e9d39f95/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 9643e87..619fc3a 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
@@ -32,12 +32,14 @@
 <upgrade xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
   <target>2.2.*.*</target>
   <order>
+  
     <group name="ZOOKEEPER" title="Zookeeper">
       <service name="ZOOKEEPER">
         <component>ZOOKEEPER_SERVER</component>
         <component>ZOOKEEPER_CLIENT</component>
       </service>
     </group>
+    
     <group name="CORE_MASTER" title="Core Masters">
       <service name="HDFS">
         <component>JOURNALNODE</component>
@@ -47,6 +49,7 @@
         <component>RESOURCEMANAGER</component>
       </service>
     </group>
+    
     <group name="CORE_SLAVES" title="Core Slaves" xsi:type="colocated">
       <service name="HDFS">
         <component>DATANODE</component>
@@ -57,15 +60,30 @@
       <service name="YARN">
         <component>NODEMANAGER</component>
       </service>
-      
       <batch>
         <percent>20</percent>
         <message>Please run additional tests</message>
       </batch>
     </group>
+    
+    <group xsi:type="cluster" name="POST_CLUSTER" title="Finalize Upgrade">
+      <execute-stage title="Confirm Finalize">
+        <task xsi:type="manual">
+          <message>Please confirm you are ready to finalize</message>
+        </task>
+      </execute-stage>
+      <execute-stage service="HDFS" component="NAMENODE" title="Execute HDFS Finalize">
+        <task xsi:type="execute">
+          <command>ls</command>
+        </task>
+      </execute-stage>
+    </group>
+    
   </order>
   
+  
   <processing>
+  
     <service name="ZOOKEEPER">
       <component name="ZOOKEEPER_SERVER">
         <!-- TODO, optimization
@@ -97,6 +115,7 @@
     </service>
     <service name="HDFS">
       <component name="NAMENODE">
+        
         <pre-upgrade>
           <!-- Backup the image,
           Enter Safemode if not already in it,

http://git-wip-us.apache.org/repos/asf/ambari/blob/e9d39f95/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/UpgradeResourceProviderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/UpgradeResourceProviderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/UpgradeResourceProviderTest.java
index 7d986c9..c55cde8 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/UpgradeResourceProviderTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/UpgradeResourceProviderTest.java
@@ -165,10 +165,9 @@ public class UpgradeResourceProviderTest {
     UpgradeEntity entity = upgrades.get(0);
     assertEquals(cluster.getClusterId(), entity.getClusterId().longValue());
 
-    assertEquals(3, entity.getUpgradeGroups().size());
-
-    UpgradeGroupEntity group = entity.getUpgradeGroups().get(0);
+    assertEquals(5, entity.getUpgradeGroups().size());
 
+    UpgradeGroupEntity group = entity.getUpgradeGroups().get(1);
     assertEquals(4, group.getItems().size());
 
     assertTrue(group.getItems().get(0).getText().contains("Preparing"));
@@ -184,16 +183,12 @@ public class UpgradeResourceProviderTest {
 
 
     List<Stage> stages = am.getRequestStatus(requests.get(0).longValue());
-    assertEquals(4, stages.size());
-    for (int i = 0; i < stages.size(); i++) {
-      Stage stage = stages.get(i);
-      UpgradeItemEntity upgradeItem = group.getItems().get(i);
-      assertEquals(stage.getStageId(), upgradeItem.getStageId().longValue());
-      assertEquals(UpgradeState.NONE, upgradeItem.getState());
-    }
+
+    assertEquals(7, stages.size());
 
     List<HostRoleCommand> tasks = am.getRequestTasks(requests.get(0).longValue());
-    assertEquals(4, tasks.size());
+    // same number of tasks as stages here
+    assertEquals(7, tasks.size());
 
     return status;
   }
@@ -240,7 +235,7 @@ public class UpgradeResourceProviderTest {
     ResourceProvider upgradeGroupResourceProvider = new UpgradeGroupResourceProvider(amc);
     resources = upgradeGroupResourceProvider.getResources(request, predicate);
 
-    assertEquals(3, resources.size());
+    assertEquals(5, resources.size());
     res = resources.iterator().next();
     assertNotNull(res.getPropertyValue("UpgradeGroup/status"));
     assertNotNull(res.getPropertyValue("UpgradeGroup/group_id"));
@@ -259,7 +254,7 @@ public class UpgradeResourceProviderTest {
     ResourceProvider upgradeItemResourceProvider = new UpgradeItemResourceProvider(amc);
     resources = upgradeItemResourceProvider.getResources(request, predicate);
 
-    assertEquals(4, resources.size());
+    assertEquals(2, resources.size());
     res = resources.iterator().next();
     assertNotNull(res.getPropertyValue("UpgradeItem/status"));
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/e9d39f95/ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeHelperTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeHelperTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeHelperTest.java
index 3ff9942..88b6125 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeHelperTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeHelperTest.java
@@ -77,14 +77,18 @@ public class UpgradeHelperTest {
     UpgradeHelper helper = new UpgradeHelper();
     List<UpgradeGroupHolder> groups = helper.createUpgrade(cluster, upgrade);
 
-    assertEquals(3, groups.size());
-    assertEquals("ZOOKEEPER", groups.get(0).name);
-    assertEquals("CORE_MASTER", groups.get(1).name);
-    assertEquals("CORE_SLAVES", groups.get(2).name);
-
-    assertEquals(6, groups.get(0).items.size());
-    assertEquals(5, groups.get(1).items.size());
-    assertEquals(6, groups.get(2).items.size());
+    assertEquals(5, groups.size());
+
+    assertEquals("PRE_CLUSTER", groups.get(0).name);
+    assertEquals("ZOOKEEPER", groups.get(1).name);
+    assertEquals("CORE_MASTER", groups.get(2).name);
+    assertEquals("CORE_SLAVES", groups.get(3).name);
+    assertEquals("POST_CLUSTER", groups.get(4).name);
+
+    assertEquals(6, groups.get(1).items.size());
+    assertEquals(5, groups.get(2).items.size());
+    assertEquals(6, groups.get(3).items.size());
+
   }
 
   public Cluster makeCluster() throws AmbariException {

http://git-wip-us.apache.org/repos/asf/ambari/blob/e9d39f95/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
index 5a6dc92..e93e985 100644
--- 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
@@ -17,7 +17,26 @@
 -->
 <upgrade xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
   <target>2.2.*</target>
+  
   <order>
+    <group xsi:type="cluster" name="PRE_CLUSTER" title="Pre Upgrade" stage="pre">
+      <execute-stage title="Confirm 1">
+        <task xsi:type="manual">
+          <message>Foo</message>
+        </task>
+      </execute-stage>
+      <execute-stage service="HDFS" component="NAMENODE" title="Finalize HDFS">
+        <task xsi:type="execute">
+          <command>ls</command>
+        </task>
+      </execute-stage>
+      <execute-stage title="Confirm 2">
+        <task xsi:type="manual">
+          <message>Foo</message>
+        </task>
+      </execute-stage>
+    </group>
+  
     <group name="ZOOKEEPER" title="Zookeeper">
       <service name="ZOOKEEPER">
         <component>ZOOKEEPER_SERVER</component>
@@ -49,9 +68,25 @@
         <message>Please run additional tests</message>
       </batch>
     </group>
+    
+    <group xsi:type="cluster" name="POST_CLUSTER" title="Finalize Upgrade" stage="post">
+      <execute-stage title="Confirm Finalize">
+        <task xsi:type="manual">
+          <message>Please confirm you are ready to finalize</message>
+        </task>
+      </execute-stage>
+      <execute-stage service="HDFS" component="NAMENODE" title="Execute HDFS Finalize">
+        <task xsi:type="execute">
+          <command>ls</command>
+        </task>
+      </execute-stage>
+    </group>
+        
   </order>
+  
 
   <processing>
+  
     <service name="ZOOKEEPER">
       <component name="ZOOKEEPER_SERVER">
         <pre-upgrade>