You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by tb...@apache.org on 2015/01/28 14:31:14 UTC

ambari git commit: AMBARI-9362 - Request status may be FAILED if underlying skippable stage is FAILED. (tbeerbower)

Repository: ambari
Updated Branches:
  refs/heads/trunk c0ed966ff -> b11fe8810


AMBARI-9362 - Request status may be FAILED if underlying skippable stage is FAILED. (tbeerbower)


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

Branch: refs/heads/trunk
Commit: b11fe88106811cb7ae7a073191028f61ce680db0
Parents: c0ed966
Author: tbeerbower <tb...@hortonworks.com>
Authored: Wed Jan 28 08:30:52 2015 -0500
Committer: tbeerbower <tb...@hortonworks.com>
Committed: Wed Jan 28 08:31:06 2015 -0500

----------------------------------------------------------------------
 .../controller/internal/CalculatedStatus.java   | 286 +++++++++++
 .../internal/RequestResourceProvider.java       |  25 +-
 .../internal/StageResourceProvider.java         | 106 +---
 .../internal/UpgradeGroupResourceProvider.java  |  70 +--
 .../apache/ambari/server/orm/dao/StageDAO.java  |  21 +
 .../internal/CalculatedStatusTest.java          | 512 +++++++++++++++++++
 .../internal/RequestResourceProviderTest.java   | 149 ++++--
 .../internal/StageResourceProviderTest.java     | 138 -----
 8 files changed, 966 insertions(+), 341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/b11fe881/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/CalculatedStatus.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/CalculatedStatus.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/CalculatedStatus.java
new file mode 100644
index 0000000..28e0d17
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/CalculatedStatus.java
@@ -0,0 +1,286 @@
+/**
+ * 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.controller.internal;
+
+import org.apache.ambari.server.actionmanager.HostRoleCommand;
+import org.apache.ambari.server.actionmanager.HostRoleStatus;
+import org.apache.ambari.server.actionmanager.Stage;
+import org.apache.ambari.server.orm.entities.HostRoleCommandEntity;
+import org.apache.ambari.server.orm.entities.StageEntity;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+
+/**
+ * Status of a request resource, calculated from a set of tasks or stages.
+ */
+public class CalculatedStatus {
+
+  /**
+   * The calculated overall status.
+   */
+  private final HostRoleStatus status;
+
+  /**
+   * The calculated percent complete.
+   */
+  private final double percent;
+
+
+  // ----- Constructors ------------------------------------------------------
+
+  /**
+   * Constructor.
+   *
+   * @param status   the calculated overall status
+   * @param percent  the calculated percent complete
+   */
+  private CalculatedStatus(HostRoleStatus status, double percent) {
+    this.status  = status;
+    this.percent = percent;
+  }
+
+
+  // ----- CalculatedStatus --------------------------------------------------
+
+  /**
+   * Get the calculated status.
+   *
+   * @return the status
+   */
+  public HostRoleStatus getStatus() {
+    return status;
+  }
+
+  /**
+   * Get the calculated percent complete.
+   *
+   * @return the percent complete
+   */
+  public double getPercent() {
+    return percent;
+  }
+
+
+  // ----- helper methods ----------------------------------------------------
+
+  /**
+   * Factory method to create a calculated status.  Calculate request status from the given
+   * collection of task entities.
+   *
+   * @param tasks      the collection of task entities
+   * @param skippable  true if a single failed status should NOT result in an overall failed status
+   *
+   * @return a calculated status
+   */
+  public static CalculatedStatus statusFromTaskEntities(Collection<HostRoleCommandEntity> tasks, boolean skippable) {
+
+    int size = tasks.size();
+
+    Map<HostRoleStatus, Integer> taskStatusCounts = CalculatedStatus.calculateTaskEntityStatusCounts(tasks);
+
+    HostRoleStatus status = calculateSummaryStatus(taskStatusCounts, size, skippable);
+
+    double progressPercent = calculateProgressPercent(taskStatusCounts, size);
+
+    return new CalculatedStatus(status, progressPercent);
+  }
+
+  /**
+   * Factory method to create a calculated status.  Calculate request status from the given
+   * collection of stage entities.
+   *
+   * @param stages  the collection of stage entities
+   *
+   * @return a calculated status
+   */
+  public static CalculatedStatus statusFromStageEntities(Collection<StageEntity> stages) {
+
+    Collection<HostRoleStatus> stageStatuses = new HashSet<HostRoleStatus>();
+    Collection<HostRoleCommandEntity> tasks = new HashSet<HostRoleCommandEntity>();
+
+    for (StageEntity stage : stages) {
+      // get all the tasks for the stage
+      Collection<HostRoleCommandEntity> stageTasks = stage.getHostRoleCommands();
+
+      // calculate the stage status from the task status counts
+      HostRoleStatus stageStatus =
+          calculateSummaryStatus(calculateTaskEntityStatusCounts(stageTasks), stageTasks.size(), stage.isSkippable());
+
+      stageStatuses.add(stageStatus);
+
+      // keep track of all of the tasks for all stages
+      tasks.addAll(stageTasks);
+    }
+
+    // calculate the overall status from the stage statuses
+    HostRoleStatus status = calculateSummaryStatus(calculateStatusCounts(stageStatuses), stageStatuses.size(), false);
+
+    // calculate the progress from the task status counts
+    double progressPercent = calculateProgressPercent(calculateTaskEntityStatusCounts(tasks), tasks.size());
+
+    return new CalculatedStatus(status, progressPercent);
+  }
+
+  /**
+   * Factory method to create a calculated status.  Calculate request status from the given
+   * collection of stages.
+   *
+   * @param stages  the collection of stages
+   *
+   * @return a calculated status
+   */
+  public static CalculatedStatus statusFromStages(Collection<Stage> stages) {
+
+    Collection<HostRoleStatus> stageStatuses = new HashSet<HostRoleStatus>();
+    Collection<HostRoleCommand> tasks = new HashSet<HostRoleCommand>();
+
+    for (Stage stage : stages) {
+      // get all the tasks for the stage
+      Collection<HostRoleCommand> stageTasks = stage.getOrderedHostRoleCommands();
+
+      // calculate the stage status from the task status counts
+      HostRoleStatus stageStatus =
+          calculateSummaryStatus(calculateTaskStatusCounts(stageTasks), stageTasks.size(), stage.isSkippable());
+
+      stageStatuses.add(stageStatus);
+
+      // keep track of all of the tasks for all stages
+      tasks.addAll(stageTasks);
+    }
+
+    // calculate the overall status from the stage statuses
+    HostRoleStatus status = calculateSummaryStatus(calculateStatusCounts(stageStatuses), stageStatuses.size(), false);
+
+    // calculate the progress from the task status counts
+    double progressPercent = calculateProgressPercent(calculateTaskStatusCounts(tasks), tasks.size());
+
+    return new CalculatedStatus(status, progressPercent);
+  }
+
+  /**
+   * Returns counts of tasks that are in various states.
+   *
+   * @param hostRoleStatuses  the collection of tasks
+   *
+   * @return a map of counts of tasks keyed by the task status
+   */
+  public static Map<HostRoleStatus, Integer> calculateStatusCounts(Collection<HostRoleStatus> hostRoleStatuses) {
+    Map<HostRoleStatus, Integer> counters = new HashMap<HostRoleStatus, Integer>();
+    // initialize
+    for (HostRoleStatus hostRoleStatus : HostRoleStatus.values()) {
+      counters.put(hostRoleStatus, 0);
+    }
+    // calculate counts
+    for (HostRoleStatus status : hostRoleStatuses) {
+      // count tasks where isCompletedState() == true as COMPLETED
+      // but don't count tasks with COMPLETED status twice
+      if (status.isCompletedState() && status != HostRoleStatus.COMPLETED) {
+        // Increase total number of completed tasks;
+        counters.put(HostRoleStatus.COMPLETED, counters.get(HostRoleStatus.COMPLETED) + 1);
+      }
+      // Increment counter for particular status
+      counters.put(status, counters.get(status) + 1);
+    }
+
+    // We overwrite the value to have the sum converged
+    counters.put(HostRoleStatus.IN_PROGRESS,
+        hostRoleStatuses.size() -
+            counters.get(HostRoleStatus.COMPLETED) -
+            counters.get(HostRoleStatus.QUEUED) -
+            counters.get(HostRoleStatus.PENDING));
+
+    return counters;
+  }
+
+  /**
+   * Returns counts of task entities that are in various states.
+   *
+   * @param tasks  the collection of task entities
+   *
+   * @return a map of counts of tasks keyed by the task status
+   */
+  private static Map<HostRoleStatus, Integer> calculateTaskEntityStatusCounts(Collection<HostRoleCommandEntity> tasks) {
+    Collection<HostRoleStatus> hostRoleStatuses = new LinkedList<HostRoleStatus>();
+
+    for (HostRoleCommandEntity hostRoleCommand : tasks) {
+      hostRoleStatuses.add(hostRoleCommand.getStatus());
+    }
+    return calculateStatusCounts(hostRoleStatuses);
+  }
+
+  /**
+   * Returns counts of tasks that are in various states.
+   *
+   * @param tasks  the collection of tasks
+   *
+   * @return a map of counts of tasks keyed by the task status
+   */
+  private static Map<HostRoleStatus, Integer> calculateTaskStatusCounts(Collection<HostRoleCommand> tasks) {
+    Collection<HostRoleStatus> hostRoleStatuses = new LinkedList<HostRoleStatus>();
+
+    for (HostRoleCommand hostRoleCommand : tasks) {
+      hostRoleStatuses.add(hostRoleCommand.getStatus());
+    }
+    return calculateStatusCounts(hostRoleStatuses);
+  }
+
+  /**
+   * Calculate the percent complete based on the given status counts.
+   *
+   * @param counters  counts of resources that are in various states
+   * @param total     total number of resources in request
+   *
+   * @return the percent complete for the stage
+   */
+  private static double calculateProgressPercent(Map<HostRoleStatus, Integer> counters, double total) {
+    return total == 0 ? 0 :
+        ((counters.get(HostRoleStatus.QUEUED)              * 0.09 +
+          counters.get(HostRoleStatus.IN_PROGRESS)         * 0.35 +
+          counters.get(HostRoleStatus.HOLDING)             * 0.35 +
+          counters.get(HostRoleStatus.HOLDING_FAILED)      * 0.35 +
+          counters.get(HostRoleStatus.HOLDING_TIMEDOUT)    * 0.35 +
+          counters.get(HostRoleStatus.COMPLETED)) / total) * 100.0;
+  }
+
+  /**
+   * Calculate an overall status based on the given status counts.
+   *
+   * @param counters   counts of resources that are in various states
+   * @param total      total number of resources in request
+   * @param skippable  true if a single failed status should NOT result in an overall failed status return
+   *
+   * @return summary request status based on statuses of tasks in different states.
+   */
+  private static HostRoleStatus calculateSummaryStatus(Map<HostRoleStatus, Integer> counters,
+                                                      int total,
+                                                      boolean skippable) {
+
+    return counters.get(HostRoleStatus.PENDING) == total ? HostRoleStatus.PENDING :
+        counters.get(HostRoleStatus.HOLDING) > 0 ? HostRoleStatus.HOLDING :
+        counters.get(HostRoleStatus.HOLDING_FAILED) > 0 ? HostRoleStatus.HOLDING_FAILED :
+        counters.get(HostRoleStatus.HOLDING_TIMEDOUT) > 0 ? HostRoleStatus.HOLDING_TIMEDOUT :
+        counters.get(HostRoleStatus.FAILED) > 0 && !skippable ? HostRoleStatus.FAILED :
+        counters.get(HostRoleStatus.ABORTED) > 0 && !skippable ? HostRoleStatus.ABORTED :
+        counters.get(HostRoleStatus.TIMEDOUT) > 0 && !skippable ? HostRoleStatus.TIMEDOUT :
+        counters.get(HostRoleStatus.COMPLETED) == total ? HostRoleStatus.COMPLETED : HostRoleStatus.IN_PROGRESS;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/b11fe881/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/RequestResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/RequestResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/RequestResourceProvider.java
index c0fdd1d..64d7ac3 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/RequestResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/RequestResourceProvider.java
@@ -21,6 +21,7 @@ import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.actionmanager.HostRoleCommand;
 import org.apache.ambari.server.actionmanager.HostRoleStatus;
+import org.apache.ambari.server.actionmanager.Stage;
 import org.apache.ambari.server.api.services.BaseRequest;
 import org.apache.ambari.server.controller.AmbariManagementController;
 import org.apache.ambari.server.controller.ExecuteActionRequest;
@@ -219,17 +220,16 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
         throw new IllegalArgumentException("Abort reason can not be empty.");
       }
 
-      List<HostRoleCommand> commands = internalRequest.getCommands();
-      HostRoleStatus internalRequestStatus =
-          StageResourceProvider.calculateSummaryStatus(
-              StageResourceProvider.calculateTaskStatusCounts(getHostRoleStatuses(commands)), commands.size(), true);
-
       if (updateRequest.getStatus() != HostRoleStatus.ABORTED) {
         throw new IllegalArgumentException(
                 String.format("%s is wrong value. The only allowed value " +
                                 "for updating request status is ABORTED",
                         updateRequest.getStatus()));
       }
+
+      HostRoleStatus internalRequestStatus =
+          CalculatedStatus.statusFromStages(internalRequest.getStages()).getStatus();
+
       if (internalRequestStatus.isCompletedState()) {
         throw new IllegalArgumentException(
                 String.format("Can not set request that is in %s state to %s state.",
@@ -465,18 +465,18 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
     }
 
     Collection<HostRoleCommand> commands = request.getCommands();
+    Collection<Stage> stages = request.getStages();
 
-    int taskCount = commands.size();
+    CalculatedStatus status = CalculatedStatus.statusFromStages(stages);
 
-    Map<HostRoleStatus, Integer> hostRoleStatusCounters =
-        StageResourceProvider.calculateTaskStatusCounts(getHostRoleStatuses(commands));
+    setResourceProperty(resource, REQUEST_STATUS_PROPERTY_ID, status.getStatus().toString(), requestedPropertyIds);
+    setResourceProperty(resource, REQUEST_PROGRESS_PERCENT_ID, status.getPercent(), requestedPropertyIds);
 
-    HostRoleStatus requestStatus =
-        StageResourceProvider.calculateSummaryStatus(hostRoleStatusCounters, taskCount, true);
+    int taskCount = commands.size();
 
-    double progressPercent = StageResourceProvider.calculateProgressPercent(hostRoleStatusCounters, taskCount);
+    Map<HostRoleStatus, Integer> hostRoleStatusCounters =
+        CalculatedStatus.calculateStatusCounts(getHostRoleStatuses(commands));
 
-    setResourceProperty(resource, REQUEST_STATUS_PROPERTY_ID, requestStatus.toString(), requestedPropertyIds);
     setResourceProperty(resource, REQUEST_TASK_CNT_ID, taskCount, requestedPropertyIds);
     setResourceProperty(resource, REQUEST_FAILED_TASK_CNT_ID,
             hostRoleStatusCounters.get(HostRoleStatus.FAILED), requestedPropertyIds);
@@ -488,7 +488,6 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
             hostRoleStatusCounters.get(HostRoleStatus.QUEUED), requestedPropertyIds);
     setResourceProperty(resource, REQUEST_COMPLETED_TASK_CNT_ID,
             hostRoleStatusCounters.get(HostRoleStatus.COMPLETED), requestedPropertyIds);
-    setResourceProperty(resource, REQUEST_PROGRESS_PERCENT_ID, progressPercent, requestedPropertyIds);
 
     return resource;
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/b11fe881/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StageResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StageResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StageResourceProvider.java
index 54e65fa..ce427a7 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StageResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StageResourceProvider.java
@@ -47,7 +47,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -265,9 +264,7 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
   private static void updateStageStatus(StageEntity entity, HostRoleStatus desiredStatus) {
     Collection<HostRoleCommandEntity> tasks = entity.getHostRoleCommands();
 
-    Map<HostRoleStatus, Integer> taskStatusCounts = calculateTaskStatusCounts(getHostRoleStatuses(tasks));
-
-    HostRoleStatus currentStatus = calculateSummaryStatus(taskStatusCounts, tasks.size(), !entity.isSkippable());
+    HostRoleStatus currentStatus = CalculatedStatus.statusFromTaskEntities(tasks, entity.isSkippable()).getStatus();
 
     if (!isValidManualTransition(currentStatus, desiredStatus)) {
       throw new IllegalArgumentException("Can not transition a stage from " +
@@ -333,110 +330,15 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
     setResourceProperty(resource, STAGE_START_TIME, startTime, requestedIds);
     setResourceProperty(resource, STAGE_END_TIME, endTime, requestedIds);
 
-    int taskCount = tasks.size();
-
-    Map<HostRoleStatus, Integer> taskStatusCounts = calculateTaskStatusCounts(getHostRoleStatuses(tasks));
+    CalculatedStatus status = CalculatedStatus.statusFromTaskEntities(tasks, entity.isSkippable());
 
-    setResourceProperty(resource, STAGE_PROGRESS_PERCENT, calculateProgressPercent(taskStatusCounts, taskCount),
-        requestedIds);
-    setResourceProperty(resource, STAGE_STATUS,
-        calculateSummaryStatus(taskStatusCounts, taskCount, !entity.isSkippable()).toString(),
-        requestedIds);
+    setResourceProperty(resource, STAGE_PROGRESS_PERCENT, status.getPercent(), requestedIds);
+    setResourceProperty(resource, STAGE_STATUS, status.getStatus().toString(), requestedIds);
 
     return resource;
   }
 
   /**
-   * Calculate the percent complete based on the given status counts.
-   *
-   * @param counters  counts of resources that are in various states
-   * @param total     total number of resources in request
-   *
-   * @return the percent complete for the stage
-   */
-  protected static double calculateProgressPercent(Map<HostRoleStatus, Integer> counters, double total) {
-    return ((counters.get(HostRoleStatus.QUEUED)      * 0.09 +
-        counters.get(HostRoleStatus.IN_PROGRESS)      * 0.35 +
-        counters.get(HostRoleStatus.HOLDING)          * 0.35 +
-        counters.get(HostRoleStatus.HOLDING_FAILED)   * 0.35 +
-        counters.get(HostRoleStatus.HOLDING_TIMEDOUT) * 0.35 +
-        counters.get(HostRoleStatus.COMPLETED)) / total) * 100.0;
-  }
-
-  /**
-   * Calculate an overall status based on the given status counts.
-   *
-   * @param counters  counts of resources that are in various states
-   * @param total     total number of resources in request
-   * @param failAll   true if a single failed status should result in an overall failed status return
-   *
-   * @return summary request status based on statuses of tasks in different states.
-   */
-  protected static HostRoleStatus calculateSummaryStatus(Map<HostRoleStatus, Integer> counters, int total,
-                                                         boolean failAll) {
-    return counters.get(HostRoleStatus.PENDING) == total ? HostRoleStatus.PENDING :
-        counters.get(HostRoleStatus.HOLDING) > 0 ? HostRoleStatus.HOLDING :
-        counters.get(HostRoleStatus.HOLDING_FAILED) > 0 ? HostRoleStatus.HOLDING_FAILED :
-        counters.get(HostRoleStatus.HOLDING_TIMEDOUT) > 0 ? HostRoleStatus.HOLDING_TIMEDOUT :
-        counters.get(HostRoleStatus.FAILED) > 0 && failAll ? HostRoleStatus.FAILED :
-        counters.get(HostRoleStatus.ABORTED) > 0 ? HostRoleStatus.ABORTED :
-        counters.get(HostRoleStatus.TIMEDOUT) > 0 && failAll ? HostRoleStatus.TIMEDOUT :
-        counters.get(HostRoleStatus.COMPLETED) == total ? HostRoleStatus.COMPLETED :
-            HostRoleStatus.IN_PROGRESS;
-  }
-
-  /**
-   * Get a collection of statuses from the given collection of task entities.
-   *
-   * @param tasks  the task entities
-   *
-   * @return a collection of statuses
-   */
-  private static Collection<HostRoleStatus> getHostRoleStatuses(Collection<HostRoleCommandEntity> tasks) {
-    Collection<HostRoleStatus> hostRoleStatuses = new LinkedList<HostRoleStatus>();
-
-    for (HostRoleCommandEntity hostRoleCommand : tasks) {
-      hostRoleStatuses.add(hostRoleCommand.getStatus());
-    }
-    return hostRoleStatuses;
-  }
-
-  /**
-   * Returns counts of tasks that are in various states.
-   *
-   * @param hostRoleStatuses  the collection of tasks
-   *
-   * @return a map of counts of tasks keyed by the task status
-   */
-  protected static Map<HostRoleStatus, Integer> calculateTaskStatusCounts(Collection<HostRoleStatus> hostRoleStatuses) {
-    Map<HostRoleStatus, Integer> counters = new HashMap<HostRoleStatus, Integer>();
-    // initialize
-    for (HostRoleStatus hostRoleStatus : HostRoleStatus.values()) {
-      counters.put(hostRoleStatus, 0);
-    }
-    // calculate counts
-    for (HostRoleStatus status : hostRoleStatuses) {
-      // count tasks where isCompletedState() == true as COMPLETED
-      // but don't count tasks with COMPLETED status twice
-      if (status.isCompletedState() && status != HostRoleStatus.COMPLETED) {
-        // Increase total number of completed tasks;
-        counters.put(HostRoleStatus.COMPLETED, counters.get(HostRoleStatus.COMPLETED) + 1);
-      }
-      // Increment counter for particular status
-      counters.put(status, counters.get(status) + 1);
-    }
-
-    // We overwrite the value to have the sum converged
-    counters.put(HostRoleStatus.IN_PROGRESS,
-        hostRoleStatuses.size() -
-        counters.get(HostRoleStatus.COMPLETED) -
-        counters.get(HostRoleStatus.QUEUED) -
-        counters.get(HostRoleStatus.PENDING));
-
-    return counters;
-  }
-
-  /**
    * Ensure that cluster information is available.
    *
    * @return the clusters information

http://git-wip-us.apache.org/repos/asf/ambari/blob/b11fe881/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeGroupResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeGroupResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeGroupResourceProvider.java
index 4288627..c89d005 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeGroupResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeGroupResourceProvider.java
@@ -17,18 +17,14 @@
  */
 package org.apache.ambari.server.controller.internal;
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.ambari.server.StaticallyInject;
-import org.apache.ambari.server.actionmanager.HostRoleStatus;
 import org.apache.ambari.server.controller.AmbariManagementController;
 import org.apache.ambari.server.controller.spi.NoSuchParentResourceException;
 import org.apache.ambari.server.controller.spi.NoSuchResourceException;
@@ -39,11 +35,11 @@ import org.apache.ambari.server.controller.spi.Resource;
 import org.apache.ambari.server.controller.spi.ResourceAlreadyExistsException;
 import org.apache.ambari.server.controller.spi.SystemException;
 import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
+import org.apache.ambari.server.orm.dao.StageDAO;
 import org.apache.ambari.server.orm.dao.UpgradeDAO;
 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.state.UpgradeHelper;
 
 import com.google.inject.Inject;
 
@@ -71,10 +67,10 @@ public class UpgradeGroupResourceProvider extends AbstractControllerResourceProv
   private static UpgradeDAO m_dao = null;
 
   /**
-   * Used to generated the correct tasks and stages during an upgrade.
+   * Used for querying stage resources.
    */
   @Inject
-  private static UpgradeHelper s_upgradeHelper;
+  private static StageDAO stageDAO = null;
 
   static {
     // properties
@@ -117,7 +113,6 @@ public class UpgradeGroupResourceProvider extends AbstractControllerResourceProv
 
     for (Map<String, Object> propertyMap : getPropertyMaps(predicate)) {
       String upgradeIdStr = (String) propertyMap.get(UPGRADE_REQUEST_ID);
-      String clusterName = (String) propertyMap.get(UPGRADE_CLUSTER_NAME);
 
       if (null == upgradeIdStr || upgradeIdStr.isEmpty()) {
         throw new IllegalArgumentException("The upgrade id is required when querying for upgrades");
@@ -126,29 +121,25 @@ public class UpgradeGroupResourceProvider extends AbstractControllerResourceProv
       Long upgradeId = Long.valueOf(upgradeIdStr);
       UpgradeEntity upgrade = m_dao.findUpgradeByRequestId(upgradeId);
 
+      Long requestId = upgrade.getRequestId();
+
       List<UpgradeGroupEntity> groups = upgrade.getUpgradeGroups();
       if (null != groups) {
 
-
         for (UpgradeGroupEntity group : upgrade.getUpgradeGroups()) {
           Resource r = toResource(upgrade, group, requestPropertyIds);
 
-          List<Long> stageIds = new ArrayList<Long>();
+          Set<Long> stageIds = new HashSet<Long>();
           for (UpgradeItemEntity itemEntity : group.getItems()) {
             stageIds.add(itemEntity.getStageId());
           }
 
-          Set<Resource> stages = s_upgradeHelper.getStageResources(clusterName,
-              upgrade.getRequestId(), stageIds);
-
-          aggregate(r, stages, requestPropertyIds);
+          aggregate(r, requestId, stageIds, requestPropertyIds);
 
           results.add(r);
         }
       }
-
     }
-
     return results;
   }
 
@@ -188,51 +179,14 @@ public class UpgradeGroupResourceProvider extends AbstractControllerResourceProv
    * Aggregates status and percent complete for stages and puts the results on the upgrade group
    *
    * @param upgradeGroup  the resource representing an upgrade group
-   * @param stages        the collection of resources representing stages
+   * @param stageIds      the set of resources ids of the stages
    * @param requestedIds  the ids for the request
    */
-  private void aggregate(Resource upgradeGroup, Collection<Resource> stages, Set<String> requestedIds) {
-
-    Map<HostRoleStatus, Integer> counters =
-        StageResourceProvider.calculateTaskStatusCounts(getHostRoleStatuses(stages));
+  private void aggregate(Resource upgradeGroup, Long requestId, Set<Long> stageIds, Set<String> requestedIds) {
 
-    setResourceProperty(upgradeGroup, UPGRADE_GROUP_STATUS,
-        StageResourceProvider.calculateSummaryStatus(counters, stages.size(), false), requestedIds);
-
-    setResourceProperty(upgradeGroup, UPGRADE_GROUP_PROGRESS_PERCENT,
-        StageResourceProvider.calculateProgressPercent(counters, stages.size()), requestedIds);
-  }
-
-  /**
-   * Get a collection of statuses from the given collection of stage resources.
-   *
-   * @param stageResources  the stage resources
-   *
-   * @return a collection of statuses
-   */
-  private static Collection<HostRoleStatus> getHostRoleStatuses(Collection<Resource> stageResources) {
-    Collection<HostRoleStatus> hostRoleStatuses = new LinkedList<HostRoleStatus>();
-
-    for (Resource stage : stageResources) {
-      HostRoleStatus status = getStatus(stage);
-
-      if (status != null) {
-        hostRoleStatuses.add(status);
-      }
-    }
-    return hostRoleStatuses;
-  }
-
-  /**
-   * Get the status of the given stage resource.
-   *
-   * @param stageResource  the resource
-   *
-   * @return  the stage status
-   */
-  private static HostRoleStatus getStatus(Resource stageResource) {
-    String status = (String) stageResource.getPropertyValue(StageResourceProvider.STAGE_STATUS);
+    CalculatedStatus status = CalculatedStatus.statusFromStageEntities(stageDAO.findByStageIds(requestId, stageIds));
 
-    return status == null ? null : HostRoleStatus.valueOf(status);
+    setResourceProperty(upgradeGroup, UPGRADE_GROUP_STATUS, status.getStatus(), requestedIds);
+    setResourceProperty(upgradeGroup, UPGRADE_GROUP_PROGRESS_PERCENT, status.getPercent(), requestedIds);
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/b11fe881/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/StageDAO.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/StageDAO.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/StageDAO.java
index f1c7832..8b1cfb3 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/StageDAO.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/StageDAO.java
@@ -42,9 +42,11 @@ import javax.persistence.criteria.CriteriaQuery;
 import javax.persistence.criteria.Order;
 import javax.persistence.metamodel.SingularAttribute;
 import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Collection;
 import java.util.Map;
+import java.util.Set;
 
 @Singleton
 public class StageDAO {
@@ -93,6 +95,25 @@ public class StageDAO {
     return daoUtils.selectList(query, requestId);
   }
 
+  /**
+   * Get the list of stage entities for the given request id and stage ids.
+   *
+   * @param requestId  the request ids
+   * @param stageIds   the set of stage ids
+   *
+   * @return the set of entities for the given ids
+   */
+  public List<StageEntity> findByStageIds(Long requestId, Set<Long> stageIds) {
+    List<StageEntity> stageEntities = new LinkedList<StageEntity>();
+
+    for (StageEntity stage : findByRequestId(requestId)) {
+      if (stageIds.contains(stage.getStageId())) {
+        stageEntities.add(stage);
+      }
+    }
+    return stageEntities;
+  }
+
   @RequiresSession
   public List<StageEntity> findByCommandStatuses(Collection<HostRoleStatus> statuses) {
     TypedQuery<StageEntity> query = entityManagerProvider.get().createQuery("SELECT stage " +

http://git-wip-us.apache.org/repos/asf/ambari/blob/b11fe881/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/CalculatedStatusTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/CalculatedStatusTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/CalculatedStatusTest.java
new file mode 100644
index 0000000..ca1e583
--- /dev/null
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/CalculatedStatusTest.java
@@ -0,0 +1,512 @@
+/**
+ * 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.controller.internal;
+
+import org.apache.ambari.server.Role;
+import org.apache.ambari.server.RoleCommand;
+import org.apache.ambari.server.actionmanager.HostRoleCommand;
+import org.apache.ambari.server.actionmanager.HostRoleStatus;
+import org.apache.ambari.server.actionmanager.Stage;
+import org.apache.ambari.server.orm.entities.HostRoleCommandEntity;
+import org.apache.ambari.server.orm.entities.StageEntity;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+/**
+ * CalculatedStatus tests.
+ */
+@SuppressWarnings("unchecked")
+public class CalculatedStatusTest {
+
+  private static long taskId = 0L;
+  private static long stageId = 0L;
+
+
+  @Test
+  public void testGetStatus() throws Exception {
+    Collection<HostRoleCommandEntity> tasks =
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.PENDING,
+            HostRoleStatus.PENDING, HostRoleStatus.PENDING);
+
+    CalculatedStatus status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(HostRoleStatus.IN_PROGRESS, status.getStatus());
+  }
+
+  @Test
+  public void testGetPercent() throws Exception {
+    Collection<HostRoleCommandEntity> tasks =
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.PENDING,
+            HostRoleStatus.PENDING, HostRoleStatus.PENDING);
+
+    CalculatedStatus status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(40.0, status.getPercent(), 0.1);
+  }
+
+  @Test
+  public void testStatusFromTaskEntities() throws Exception {
+
+
+    // Pending stage
+    Collection<HostRoleCommandEntity> tasks =
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING);
+
+    CalculatedStatus status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(HostRoleStatus.PENDING, status.getStatus());
+    assertEquals(0.0, status.getPercent(), 0.1);
+
+    // failed stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.FAILED, HostRoleStatus.ABORTED,
+        HostRoleStatus.ABORTED, HostRoleStatus.ABORTED);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(HostRoleStatus.FAILED, status.getStatus());
+    assertEquals(100.0, status.getPercent(), 0.1);
+
+    // failed skippable stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.FAILED, HostRoleStatus.ABORTED,
+        HostRoleStatus.ABORTED, HostRoleStatus.ABORTED);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, true);
+
+    assertEquals(HostRoleStatus.COMPLETED, status.getStatus());
+    assertEquals(100.0, status.getPercent(), 0.1);
+
+    // timed out stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.TIMEDOUT, HostRoleStatus.ABORTED,
+        HostRoleStatus.ABORTED, HostRoleStatus.ABORTED);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(HostRoleStatus.ABORTED, status.getStatus());
+    assertEquals(100.0, status.getPercent(), 0.1);
+
+    // timed out skippable stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.TIMEDOUT, HostRoleStatus.ABORTED,
+        HostRoleStatus.ABORTED, HostRoleStatus.ABORTED);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, true);
+
+    assertEquals(HostRoleStatus.COMPLETED, status.getStatus());
+    assertEquals(100.0, status.getPercent(), 0.1);
+
+    // aborted stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.ABORTED, HostRoleStatus.ABORTED,
+        HostRoleStatus.ABORTED, HostRoleStatus.ABORTED);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(HostRoleStatus.ABORTED, status.getStatus());
+    assertEquals(100.0, status.getPercent(), 0.1);
+
+    // aborted skippable stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.ABORTED, HostRoleStatus.ABORTED,
+        HostRoleStatus.ABORTED, HostRoleStatus.ABORTED);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, true);
+
+    assertEquals(HostRoleStatus.COMPLETED, status.getStatus());
+    assertEquals(100.0, status.getPercent(), 0.1);
+
+    // in progress stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.PENDING,
+        HostRoleStatus.PENDING, HostRoleStatus.PENDING);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(HostRoleStatus.IN_PROGRESS, status.getStatus());
+    assertEquals(40.0, status.getPercent(), 0.1);
+
+    // completed stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED,
+        HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(HostRoleStatus.COMPLETED, status.getStatus());
+    assertEquals(100.0, status.getPercent(), 0.1);
+
+    // holding stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.HOLDING,
+        HostRoleStatus.PENDING, HostRoleStatus.PENDING);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(HostRoleStatus.HOLDING, status.getStatus());
+    assertEquals(54.0, status.getPercent(), 0.1);
+
+    // holding failed stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.HOLDING_FAILED,
+        HostRoleStatus.PENDING, HostRoleStatus.PENDING);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(HostRoleStatus.HOLDING_FAILED, status.getStatus());
+    assertEquals(54.0, status.getPercent(), 0.1);
+
+    // holding timed out stage
+    tasks = getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.HOLDING_TIMEDOUT,
+        HostRoleStatus.PENDING, HostRoleStatus.PENDING);
+
+    status = CalculatedStatus.statusFromTaskEntities(tasks, false);
+
+    assertEquals(HostRoleStatus.HOLDING_TIMEDOUT, status.getStatus());
+    assertEquals(54.0, status.getPercent(), 0.1);
+  }
+
+  @Test
+  public void testStatusFromStageEntities() throws Exception {
+
+    // completed request
+    Collection<StageEntity> stages = getStageEntities(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED)
+    );
+
+    CalculatedStatus status = CalculatedStatus.statusFromStageEntities(stages);
+
+    assertEquals(HostRoleStatus.COMPLETED, status.getStatus());
+    assertEquals(100.0, status.getPercent(), 0.1);
+
+    // in progress request
+    stages = getStageEntities(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.IN_PROGRESS, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStageEntities(stages);
+
+    assertEquals(HostRoleStatus.IN_PROGRESS, status.getStatus());
+    assertEquals(48.3, status.getPercent(), 0.1);
+
+    // pending request
+    stages = getStageEntities(
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStageEntities(stages);
+
+    assertEquals(HostRoleStatus.PENDING, status.getStatus());
+    assertEquals(0.0, status.getPercent(), 0.1);
+
+    // failed request
+    stages = getStageEntities(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.FAILED, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStageEntities(stages);
+
+    assertEquals(HostRoleStatus.FAILED, status.getStatus());
+    assertEquals(55.55, status.getPercent(), 0.1);
+
+    // timed out request
+    stages = getStageEntities(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.TIMEDOUT),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStageEntities(stages);
+
+    assertEquals(HostRoleStatus.TIMEDOUT, status.getStatus());
+    assertEquals(66.66, status.getPercent(), 0.1);
+
+    // holding request
+    stages = getStageEntities(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.HOLDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStageEntities(stages);
+
+    assertEquals(HostRoleStatus.HOLDING, status.getStatus());
+    assertEquals(47.5, status.getPercent(), 0.1);
+
+    // holding failed request
+    stages = getStageEntities(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.HOLDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStageEntities(stages);
+
+    assertEquals(HostRoleStatus.HOLDING, status.getStatus());
+    assertEquals(47.5, status.getPercent(), 0.1);
+
+    // holding timed out request
+    stages = getStageEntities(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.HOLDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStageEntities(stages);
+
+    assertEquals(HostRoleStatus.HOLDING, status.getStatus());
+    assertEquals(47.5, status.getPercent(), 0.1);
+  }
+
+  @Test
+  public void testStatusFromStages() throws Exception {
+    Collection<Stage> stages;
+    CalculatedStatus status;
+
+    // completed request
+    stages = getStages(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED)
+    );
+
+    status = CalculatedStatus.statusFromStages(stages);
+
+    assertEquals(HostRoleStatus.COMPLETED, status.getStatus());
+    assertEquals(100.0, status.getPercent(), 0.1);
+
+    // in progress request
+    stages = getStages(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.IN_PROGRESS, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStages(stages);
+
+    assertEquals(HostRoleStatus.IN_PROGRESS, status.getStatus());
+    assertEquals(48.3, status.getPercent(), 0.1);
+
+    // pending request
+    stages = getStages(
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStages(stages);
+
+    assertEquals(HostRoleStatus.PENDING, status.getStatus());
+    assertEquals(0.0, status.getPercent(), 0.1);
+
+    // failed request
+    stages = getStages(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.FAILED, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStages(stages);
+
+    assertEquals(HostRoleStatus.FAILED, status.getStatus());
+    assertEquals(55.55, status.getPercent(), 0.1);
+
+    // timed out request
+    stages = getStages(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.TIMEDOUT),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStages(stages);
+
+    assertEquals(HostRoleStatus.TIMEDOUT, status.getStatus());
+    assertEquals(66.66, status.getPercent(), 0.1);
+
+    // holding request
+    stages = getStages(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.HOLDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStages(stages);
+
+    assertEquals(HostRoleStatus.HOLDING, status.getStatus());
+    assertEquals(47.5, status.getPercent(), 0.1);
+
+    // holding failed request
+    stages = getStages(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.HOLDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStages(stages);
+
+    assertEquals(HostRoleStatus.HOLDING, status.getStatus());
+    assertEquals(47.5, status.getPercent(), 0.1);
+
+    // holding timed out request
+    stages = getStages(
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED),
+        getTaskEntities(HostRoleStatus.COMPLETED, HostRoleStatus.COMPLETED, HostRoleStatus.HOLDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING),
+        getTaskEntities(HostRoleStatus.PENDING, HostRoleStatus.PENDING, HostRoleStatus.PENDING)
+    );
+
+    status = CalculatedStatus.statusFromStages(stages);
+
+    assertEquals(HostRoleStatus.HOLDING, status.getStatus());
+    assertEquals(47.5, status.getPercent(), 0.1);
+  }
+
+  @Test
+  public void testCalculateStatusCounts() throws Exception {
+    Collection<HostRoleStatus> hostRoleStatuses = new LinkedList<HostRoleStatus>();
+
+    hostRoleStatuses.add(HostRoleStatus.PENDING);
+    hostRoleStatuses.add(HostRoleStatus.QUEUED);
+    hostRoleStatuses.add(HostRoleStatus.HOLDING);
+    hostRoleStatuses.add(HostRoleStatus.HOLDING_FAILED);
+    hostRoleStatuses.add(HostRoleStatus.HOLDING_TIMEDOUT);
+    hostRoleStatuses.add(HostRoleStatus.IN_PROGRESS);
+    hostRoleStatuses.add(HostRoleStatus.IN_PROGRESS);
+    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
+    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
+    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
+    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
+    hostRoleStatuses.add(HostRoleStatus.FAILED);
+    hostRoleStatuses.add(HostRoleStatus.TIMEDOUT);
+    hostRoleStatuses.add(HostRoleStatus.ABORTED);
+
+    Map<HostRoleStatus, Integer> counts = CalculatedStatus.calculateStatusCounts(hostRoleStatuses);
+
+    assertEquals(1L, (long) counts.get(HostRoleStatus.PENDING));
+    assertEquals(1L, (long) counts.get(HostRoleStatus.QUEUED));
+    assertEquals(1L, (long) counts.get(HostRoleStatus.HOLDING));
+    assertEquals(1L, (long) counts.get(HostRoleStatus.HOLDING_FAILED));
+    assertEquals(1L, (long) counts.get(HostRoleStatus.HOLDING_TIMEDOUT));
+    assertEquals(5L, (long) counts.get(HostRoleStatus.IN_PROGRESS));
+    assertEquals(7L, (long) counts.get(HostRoleStatus.COMPLETED));
+    assertEquals(1L, (long) counts.get(HostRoleStatus.FAILED));
+    assertEquals(1L, (long) counts.get(HostRoleStatus.TIMEDOUT));
+    assertEquals(1L, (long) counts.get(HostRoleStatus.ABORTED));
+  }
+
+  private Collection<HostRoleCommandEntity> getTaskEntities(HostRoleStatus... statuses) {
+    Collection<HostRoleCommandEntity> entities = new LinkedList<HostRoleCommandEntity>();
+
+    for (HostRoleStatus status : statuses) {
+      HostRoleCommandEntity entity = new HostRoleCommandEntity();
+      entity.setTaskId(taskId++);
+      entity.setStatus(status);
+
+      entities.add(entity);
+    }
+    return entities;
+  }
+
+  private Collection<StageEntity> getStageEntities(Collection<HostRoleCommandEntity> ... taskCollections) {
+
+    Collection<StageEntity> entities = new LinkedList<StageEntity>();
+
+    for (Collection<HostRoleCommandEntity> taskEntities : taskCollections) {
+      StageEntity entity = new StageEntity();
+      entity.setStageId(stageId++);
+      entity.setHostRoleCommands(taskEntities);
+
+      entities.add(entity);
+    }
+    return entities;
+  }
+
+  private Collection<Stage> getStages(Collection<HostRoleCommandEntity> ... taskCollections) {
+
+    Collection<Stage> entities = new LinkedList<Stage>();
+
+    for (Collection<HostRoleCommandEntity> taskEntities : taskCollections) {
+      TestStage stage = new TestStage();
+      stage.setStageId(stageId++);
+      stage.setHostRoleCommands(taskEntities);
+
+      entities.add(stage);
+    }
+    return entities;
+  }
+
+  private class TestStage extends Stage {
+
+    private final List<HostRoleCommand> hostRoleCommands = new LinkedList<HostRoleCommand>();
+
+    private TestStage() {
+      super(1L, "", "", 1L, "", "", "", "");
+    }
+
+    void setHostRoleCommands(Collection<HostRoleCommandEntity> tasks) {
+      for (HostRoleCommandEntity task : tasks) {
+        TestCommand command = new TestCommand(taskId++);
+        command.setStatus(task.getStatus());
+        hostRoleCommands.add(command);
+      }
+    }
+
+    @Override
+    public List<HostRoleCommand> getOrderedHostRoleCommands() {
+      return hostRoleCommands;
+    }
+  }
+
+  private class TestCommand extends HostRoleCommand {
+    private final long taskId;
+
+    public TestCommand(long taskId) {
+      super("", Role.AMBARI_SERVER_ACTION, null, RoleCommand.START);
+      this.taskId = taskId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      if (!super.equals(o)) return false;
+
+      TestCommand that = (TestCommand) o;
+
+      return taskId == that.taskId;
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = super.hashCode();
+      result = 31 * result + (int) (taskId ^ (taskId >>> 32));
+      return result;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/b11fe881/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/RequestResourceProviderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/RequestResourceProviderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/RequestResourceProviderTest.java
index 3ab609d..b4b212b 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/RequestResourceProviderTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/RequestResourceProviderTest.java
@@ -43,6 +43,7 @@ import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.actionmanager.HostRoleCommand;
 import org.apache.ambari.server.actionmanager.HostRoleStatus;
+import org.apache.ambari.server.actionmanager.Stage;
 import org.apache.ambari.server.api.services.BaseRequest;
 import org.apache.ambari.server.controller.AmbariManagementController;
 import org.apache.ambari.server.controller.ExecuteActionRequest;
@@ -126,7 +127,7 @@ public class RequestResourceProviderTest {
       managementController);
 
     Map<String, String> requestInfoProperties = new HashMap<String, String>();
-    Request request = null;
+    Request request;
     Predicate predicate = new PredicateBuilder()
       .property(RequestResourceProvider.REQUEST_CLUSTER_NAME_PROPERTY_ID)
       .equals("foo_cluster")
@@ -182,27 +183,33 @@ public class RequestResourceProviderTest {
     AmbariManagementController managementController = createMock(AmbariManagementController.class);
     ActionManager actionManager = createNiceMock(ActionManager.class);
     HostRoleCommand hostRoleCommand = createNiceMock(HostRoleCommand.class);
-
+    Stage stage = createNiceMock(Stage.class);
 
     List<HostRoleCommand> hostRoleCommands = new LinkedList<HostRoleCommand>();
     hostRoleCommands.add(hostRoleCommand);
 
+    Collection<Stage> stages = new HashSet<Stage>();
+    stages.add(stage);
+
     org.apache.ambari.server.actionmanager.Request requestMock =
         createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock.getCommands()).andReturn(hostRoleCommands).anyTimes();
+    expect(requestMock.getStages()).andReturn(stages).anyTimes();
     expect(requestMock.getRequestContext()).andReturn("this is a context").anyTimes();
     expect(requestMock.getRequestId()).andReturn(100L).anyTimes();
 
+    expect(stage.getOrderedHostRoleCommands()).andReturn(hostRoleCommands).anyTimes();
+
     Capture<Collection<Long>> requestIdsCapture = new Capture<Collection<Long>>();
 
     // set expectations
     expect(managementController.getActionManager()).andReturn(actionManager);
     expect(actionManager.getRequests(capture(requestIdsCapture))).andReturn(Collections.singletonList(requestMock)).anyTimes();
     expect(hostRoleCommand.getRequestId()).andReturn(100L).anyTimes();
-    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS);
+    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS).anyTimes();
 
     // replay
-    replay(managementController, actionManager, hostRoleCommand, requestMock);
+    replay(managementController, actionManager, hostRoleCommand, requestMock, stage);
 
     ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
         type,
@@ -227,7 +234,7 @@ public class RequestResourceProviderTest {
     }
 
     // verify
-    verify(managementController, actionManager, hostRoleCommand);
+    verify(managementController, actionManager, hostRoleCommand, stage);
   }
 
   @Test
@@ -237,18 +244,23 @@ public class RequestResourceProviderTest {
     AmbariManagementController managementController = createMock(AmbariManagementController.class);
     ActionManager actionManager = createNiceMock(ActionManager.class);
     HostRoleCommand hostRoleCommand = createNiceMock(HostRoleCommand.class);
-
+    Stage stage = createNiceMock(Stage.class);
 
     List<HostRoleCommand> hostRoleCommands = new LinkedList<HostRoleCommand>();
     hostRoleCommands.add(hostRoleCommand);
 
+    Collection<Stage> stages = new HashSet<Stage>();
+    stages.add(stage);
+
     org.apache.ambari.server.actionmanager.Request requestMock =
         createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock.getCommands()).andReturn(hostRoleCommands).anyTimes();
+    expect(requestMock.getStages()).andReturn(stages).anyTimes();
     expect(requestMock.getRequestContext()).andReturn("this is a context").anyTimes();
     expect(requestMock.getRequestId()).andReturn(100L).anyTimes();
     expect(requestMock.getRequestScheduleId()).andReturn(11L).anyTimes();
 
+    expect(stage.getOrderedHostRoleCommands()).andReturn(hostRoleCommands).anyTimes();
 
     Capture<Collection<Long>> requestIdsCapture = new Capture<Collection<Long>>();
 
@@ -256,10 +268,10 @@ public class RequestResourceProviderTest {
     expect(managementController.getActionManager()).andReturn(actionManager);
     expect(actionManager.getRequests(capture(requestIdsCapture))).andReturn(Collections.singletonList(requestMock)).anyTimes();
     expect(hostRoleCommand.getRequestId()).andReturn(100L).anyTimes();
-    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS);
+    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS).anyTimes();
 
     // replay
-    replay(managementController, actionManager, hostRoleCommand, requestMock);
+    replay(managementController, actionManager, hostRoleCommand, requestMock, stage);
 
     ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
         type,
@@ -287,7 +299,7 @@ public class RequestResourceProviderTest {
     }
 
     // verify
-    verify(managementController, actionManager, hostRoleCommand);
+    verify(managementController, actionManager, hostRoleCommand, stage);
   }
 
   @Test
@@ -297,18 +309,23 @@ public class RequestResourceProviderTest {
     AmbariManagementController managementController = createMock(AmbariManagementController.class);
     ActionManager actionManager = createNiceMock(ActionManager.class);
     HostRoleCommand hostRoleCommand = createNiceMock(HostRoleCommand.class);
-
+    Stage stage = createNiceMock(Stage.class);
 
     List<HostRoleCommand> hostRoleCommands = new LinkedList<HostRoleCommand>();
     hostRoleCommands.add(hostRoleCommand);
 
+    Collection<Stage> stages = new HashSet<Stage>();
+    stages.add(stage);
+
     org.apache.ambari.server.actionmanager.Request requestMock =
         createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock.getCommands()).andReturn(hostRoleCommands).anyTimes();
+    expect(requestMock.getStages()).andReturn(stages).anyTimes();
     expect(requestMock.getRequestContext()).andReturn("this is a context").anyTimes();
     expect(requestMock.getRequestId()).andReturn(100L).anyTimes();
     expect(requestMock.getRequestScheduleId()).andReturn(null).anyTimes();
 
+    expect(stage.getOrderedHostRoleCommands()).andReturn(hostRoleCommands).anyTimes();
 
     Capture<Collection<Long>> requestIdsCapture = new Capture<Collection<Long>>();
 
@@ -316,10 +333,10 @@ public class RequestResourceProviderTest {
     expect(managementController.getActionManager()).andReturn(actionManager);
     expect(actionManager.getRequests(capture(requestIdsCapture))).andReturn(Collections.singletonList(requestMock)).anyTimes();
     expect(hostRoleCommand.getRequestId()).andReturn(100L).anyTimes();
-    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS);
+    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS).anyTimes();
 
     // replay
-    replay(managementController, actionManager, hostRoleCommand, requestMock);
+    replay(managementController, actionManager, hostRoleCommand, requestMock, stage);
 
     ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
         type,
@@ -347,7 +364,7 @@ public class RequestResourceProviderTest {
     }
 
     // verify
-    verify(managementController, actionManager, hostRoleCommand);
+    verify(managementController, actionManager, hostRoleCommand, stage);
   }
 
   @Test
@@ -357,15 +374,20 @@ public class RequestResourceProviderTest {
     AmbariManagementController managementController = createMock(AmbariManagementController.class);
     ActionManager actionManager = createNiceMock(ActionManager.class);
     HostRoleCommand hostRoleCommand = createNiceMock(HostRoleCommand.class);
+    Stage stage = createNiceMock(Stage.class);
     Clusters clusters = createNiceMock(Clusters.class);
     Cluster cluster = createNiceMock(Cluster.class);
 
     List<HostRoleCommand> hostRoleCommands = new LinkedList<HostRoleCommand>();
     hostRoleCommands.add(hostRoleCommand);
 
+    Collection<Stage> stages = new HashSet<Stage>();
+    stages.add(stage);
+
     org.apache.ambari.server.actionmanager.Request requestMock =
         createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock.getCommands()).andReturn(hostRoleCommands).anyTimes();
+    expect(requestMock.getStages()).andReturn(stages).anyTimes();
     expect(requestMock.getRequestContext()).andReturn("this is a context").anyTimes();
     expect(requestMock.getClusterName()).andReturn("c1").anyTimes();
     expect(requestMock.getRequestId()).andReturn(100L).anyTimes();
@@ -379,10 +401,12 @@ public class RequestResourceProviderTest {
     expect(clusters.getCluster("bad-cluster")).andThrow(new AmbariException("bad cluster!")).anyTimes();
     expect(actionManager.getRequests(capture(requestIdsCapture))).andReturn(Collections.singletonList(requestMock));
     expect(hostRoleCommand.getRequestId()).andReturn(100L).anyTimes();
-    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS);
+    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS).anyTimes();
+
+    expect(stage.getOrderedHostRoleCommands()).andReturn(hostRoleCommands).anyTimes();
 
     // replay
-    replay(managementController, actionManager, hostRoleCommand, clusters, cluster, requestMock);
+    replay(managementController, actionManager, hostRoleCommand, clusters, cluster, requestMock, stage);
 
     ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
         type,
@@ -420,7 +444,7 @@ public class RequestResourceProviderTest {
     }
 
     // verify
-    verify(managementController, actionManager, hostRoleCommand, clusters, cluster);
+    verify(managementController, actionManager, hostRoleCommand, clusters, cluster, stage);
   }
 
   @Test
@@ -430,24 +454,35 @@ public class RequestResourceProviderTest {
     AmbariManagementController managementController = createMock(AmbariManagementController.class);
     ActionManager actionManager = createNiceMock(ActionManager.class);
     HostRoleCommand hostRoleCommand = createNiceMock(HostRoleCommand.class);
+    Stage stage = createNiceMock(Stage.class);
+    Stage stage2 = createNiceMock(Stage.class);
 
     List<HostRoleCommand> hostRoleCommands = new LinkedList<HostRoleCommand>();
     hostRoleCommands.add(hostRoleCommand);
 
+    Collection<Stage> stages = new HashSet<Stage>();
+    stages.add(stage);
+    stages.add(stage2);
+
     org.apache.ambari.server.actionmanager.Request requestMock =
         createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock.getCommands()).andReturn(hostRoleCommands).anyTimes();
+    expect(requestMock.getStages()).andReturn(stages).anyTimes();
     expect(requestMock.getRequestContext()).andReturn("this is a context").anyTimes();
     expect(requestMock.getRequestId()).andReturn(100L).anyTimes();
 
     org.apache.ambari.server.actionmanager.Request requestMock1 =
         createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock1.getCommands()).andReturn(hostRoleCommands).anyTimes();
+    expect(requestMock1.getStages()).andReturn(stages).anyTimes();
     expect(requestMock1.getRequestContext()).andReturn("this is a context").anyTimes();
     expect(requestMock1.getRequestId()).andReturn(101L).anyTimes();
 
     Capture<Collection<Long>> requestIdsCapture = new Capture<Collection<Long>>();
 
+    expect(stage.getOrderedHostRoleCommands()).andReturn(hostRoleCommands).anyTimes();
+    expect(stage2.getOrderedHostRoleCommands()).andReturn(hostRoleCommands).anyTimes();
+
     // set expectations
     expect(managementController.getActionManager()).andReturn(actionManager).anyTimes();
     expect(actionManager.getRequests(capture(requestIdsCapture))).
@@ -455,7 +490,7 @@ public class RequestResourceProviderTest {
     expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS).anyTimes();
 
     // replay
-    replay(managementController, actionManager, hostRoleCommand, requestMock, requestMock1);
+    replay(managementController, actionManager, hostRoleCommand, requestMock, requestMock1, stage, stage2);
 
     ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
         type,
@@ -480,7 +515,7 @@ public class RequestResourceProviderTest {
     }
 
     // verify
-    verify(managementController, actionManager, hostRoleCommand);
+    verify(managementController, actionManager, hostRoleCommand, stage, stage2);
   }
 
   @Test
@@ -493,6 +528,8 @@ public class RequestResourceProviderTest {
     HostRoleCommand hostRoleCommand1 = createNiceMock(HostRoleCommand.class);
     HostRoleCommand hostRoleCommand2 = createNiceMock(HostRoleCommand.class);
     HostRoleCommand hostRoleCommand3 = createNiceMock(HostRoleCommand.class);
+    Stage stage = createNiceMock(Stage.class);
+    Stage stage2 = createNiceMock(Stage.class);
 
     List<HostRoleCommand> hostRoleCommands0 = new LinkedList<HostRoleCommand>();
     hostRoleCommands0.add(hostRoleCommand0);
@@ -502,6 +539,10 @@ public class RequestResourceProviderTest {
     hostRoleCommands1.add(hostRoleCommand2);
     hostRoleCommands1.add(hostRoleCommand3);
 
+    Collection<Stage> stages = new HashSet<Stage>();
+    stages.add(stage);
+    stages.add(stage2);
+
     org.apache.ambari.server.actionmanager.Request requestMock0 =
         createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock0.getCommands()).andReturn(hostRoleCommands0).anyTimes();
@@ -514,6 +555,12 @@ public class RequestResourceProviderTest {
     expect(requestMock1.getRequestContext()).andReturn("this is a context").anyTimes();
     expect(requestMock1.getRequestId()).andReturn(101L).anyTimes();
 
+    expect(requestMock0.getStages()).andReturn(stages).anyTimes();
+    expect(requestMock1.getStages()).andReturn(stages).anyTimes();
+
+    expect(stage.getOrderedHostRoleCommands()).andReturn(hostRoleCommands0).anyTimes();
+    expect(stage2.getOrderedHostRoleCommands()).andReturn(hostRoleCommands1).anyTimes();
+
     Capture<Collection<Long>> requestIdsCapture = new Capture<Collection<Long>>();
 
     // set expectations
@@ -531,7 +578,7 @@ public class RequestResourceProviderTest {
 
     // replay
     replay(managementController, actionManager, hostRoleCommand0, hostRoleCommand1, hostRoleCommand2, hostRoleCommand3,
-        requestMock0, requestMock1);
+        requestMock0, requestMock1, stage, stage2);
 
     ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
         type,
@@ -566,7 +613,7 @@ public class RequestResourceProviderTest {
     }
 
     // verify
-    verify(managementController, actionManager, hostRoleCommand0, hostRoleCommand1, hostRoleCommand2, hostRoleCommand3);
+    verify(managementController, actionManager, hostRoleCommand0, hostRoleCommand1, hostRoleCommand2, hostRoleCommand3, stage, stage2);
   }
 
   @Test
@@ -579,6 +626,8 @@ public class RequestResourceProviderTest {
     HostRoleCommand hostRoleCommand1 = createNiceMock(HostRoleCommand.class);
     HostRoleCommand hostRoleCommand2 = createNiceMock(HostRoleCommand.class);
     HostRoleCommand hostRoleCommand3 = createNiceMock(HostRoleCommand.class);
+    Stage stage = createNiceMock(Stage.class);
+    Stage stage2 = createNiceMock(Stage.class);
 
     List<HostRoleCommand> hostRoleCommands0 = new LinkedList<HostRoleCommand>();
     hostRoleCommands0.add(hostRoleCommand0);
@@ -588,6 +637,12 @@ public class RequestResourceProviderTest {
     hostRoleCommands1.add(hostRoleCommand2);
     hostRoleCommands1.add(hostRoleCommand3);
 
+    Collection<Stage> stages = new HashSet<Stage>();
+    stages.add(stage);
+
+    Collection<Stage> stages2 = new HashSet<Stage>();
+    stages2.add(stage2);
+
     org.apache.ambari.server.actionmanager.Request requestMock0 =
         createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock0.getCommands()).andReturn(hostRoleCommands0).anyTimes();
@@ -600,6 +655,12 @@ public class RequestResourceProviderTest {
     expect(requestMock1.getRequestContext()).andReturn("this is a context").anyTimes();
     expect(requestMock1.getRequestId()).andReturn(101L).anyTimes();
 
+    expect(requestMock0.getStages()).andReturn(stages).anyTimes();
+    expect(requestMock1.getStages()).andReturn(stages2).anyTimes();
+
+    expect(stage.getOrderedHostRoleCommands()).andReturn(hostRoleCommands0).anyTimes();
+    expect(stage2.getOrderedHostRoleCommands()).andReturn(hostRoleCommands1).anyTimes();
+
     Capture<Collection<Long>> requestIdsCapture = new Capture<Collection<Long>>();
 
     // set expectations
@@ -617,7 +678,7 @@ public class RequestResourceProviderTest {
 
     // replay
     replay(managementController, actionManager, hostRoleCommand0, hostRoleCommand1, hostRoleCommand2, hostRoleCommand3,
-        requestMock0, requestMock1);
+        requestMock0, requestMock1, stage, stage2);
 
     ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
         type,
@@ -661,7 +722,7 @@ public class RequestResourceProviderTest {
     }
 
     // verify
-    verify(managementController, actionManager, hostRoleCommand0, hostRoleCommand1, hostRoleCommand2, hostRoleCommand3);
+    verify(managementController, actionManager, hostRoleCommand0, hostRoleCommand1, hostRoleCommand2, hostRoleCommand3, stage, stage2);
   }
 
   @Test
@@ -674,6 +735,8 @@ public class RequestResourceProviderTest {
     HostRoleCommand hostRoleCommand1 = createNiceMock(HostRoleCommand.class);
     HostRoleCommand hostRoleCommand2 = createNiceMock(HostRoleCommand.class);
     HostRoleCommand hostRoleCommand3 = createNiceMock(HostRoleCommand.class);
+    Stage stage = createNiceMock(Stage.class);
+    Stage stage2 = createNiceMock(Stage.class);
 
     List<HostRoleCommand> hostRoleCommands0 = new LinkedList<HostRoleCommand>();
     hostRoleCommands0.add(hostRoleCommand0);
@@ -683,6 +746,12 @@ public class RequestResourceProviderTest {
     hostRoleCommands1.add(hostRoleCommand2);
     hostRoleCommands1.add(hostRoleCommand3);
 
+    Collection<Stage> stages = new HashSet<Stage>();
+    stages.add(stage);
+
+    Collection<Stage> stages2 = new HashSet<Stage>();
+    stages2.add(stage2);
+
     org.apache.ambari.server.actionmanager.Request requestMock0 =
         createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock0.getCommands()).andReturn(hostRoleCommands0).anyTimes();
@@ -695,6 +764,12 @@ public class RequestResourceProviderTest {
     expect(requestMock1.getRequestContext()).andReturn("this is a context").anyTimes();
     expect(requestMock1.getRequestId()).andReturn(101L).anyTimes();
 
+    expect(requestMock0.getStages()).andReturn(stages).anyTimes();
+    expect(requestMock1.getStages()).andReturn(stages2).anyTimes();
+
+    expect(stage.getOrderedHostRoleCommands()).andReturn(hostRoleCommands0).anyTimes();
+    expect(stage2.getOrderedHostRoleCommands()).andReturn(hostRoleCommands1).anyTimes();
+
     Capture<Collection<Long>> requestIdsCapture = new Capture<Collection<Long>>();
 
     // set expectations
@@ -712,7 +787,7 @@ public class RequestResourceProviderTest {
 
     // replay
     replay(managementController, actionManager, hostRoleCommand0, hostRoleCommand1, hostRoleCommand2, hostRoleCommand3,
-        requestMock0, requestMock1);
+        requestMock0, requestMock1, stage, stage2);
 
     ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
         type,
@@ -757,7 +832,7 @@ public class RequestResourceProviderTest {
     }
 
     // verify
-    verify(managementController, actionManager, hostRoleCommand0, hostRoleCommand1, hostRoleCommand2, hostRoleCommand3);
+    verify(managementController, actionManager, hostRoleCommand0, hostRoleCommand1, hostRoleCommand2, hostRoleCommand3, stage, stage2);
   }
 
   @Test
@@ -767,14 +842,21 @@ public class RequestResourceProviderTest {
     AmbariManagementController managementController = createMock(AmbariManagementController.class);
     ActionManager actionManager = createNiceMock(ActionManager.class);
     HostRoleCommand hostRoleCommand = createNiceMock(HostRoleCommand.class);
+    Stage stage = createNiceMock(Stage.class);
     Clusters clusters = createNiceMock(Clusters.class);
 
     List<HostRoleCommand> hostRoleCommands = new LinkedList<HostRoleCommand>();
     hostRoleCommands.add(hostRoleCommand);
 
+    Collection<Stage> stages = new HashSet<Stage>();
+    stages.add(stage);
+
     org.apache.ambari.server.actionmanager.Request requestMock =
             createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock.getCommands()).andReturn(hostRoleCommands).anyTimes();
+    expect(requestMock.getStages()).andReturn(stages).anyTimes();
+
+    expect(stage.getOrderedHostRoleCommands()).andReturn(hostRoleCommands).anyTimes();
 
     Capture<Collection<Long>> requestIdsCapture = new Capture<Collection<Long>>();
 
@@ -787,7 +869,7 @@ public class RequestResourceProviderTest {
     RequestStatusResponse response = createNiceMock(RequestStatusResponse.class);
 
     // replay
-    replay(managementController, actionManager, hostRoleCommand, clusters, requestMock, response);
+    replay(managementController, actionManager, hostRoleCommand, clusters, requestMock, response, stage);
 
     ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
         type,
@@ -858,7 +940,7 @@ public class RequestResourceProviderTest {
       }
     }
     // verify
-    verify(managementController, response);
+    verify(managementController, response, stage);
   }
 
   @Test
@@ -1158,7 +1240,7 @@ public class RequestResourceProviderTest {
     ExecuteActionRequest capturedRequest = actionRequest.getValue();
 
     Assert.assertTrue(actionRequest.hasCaptured());
-    Assert.assertFalse("expected an action", Boolean.valueOf(capturedRequest.isCommand()));
+    Assert.assertFalse("expected an action", capturedRequest.isCommand());
     Assert.assertEquals("check_java", capturedRequest.getActionName());
     Assert.assertEquals(null, capturedRequest.getCommandName());
     Assert.assertNotNull(capturedRequest.getResourceFilters());
@@ -1178,18 +1260,25 @@ public class RequestResourceProviderTest {
     AmbariManagementController managementController = createMock(AmbariManagementController.class);
     ActionManager actionManager = createNiceMock(ActionManager.class);
     HostRoleCommand hostRoleCommand = createNiceMock(HostRoleCommand.class);
+    Stage stage = createNiceMock(Stage.class);
     Clusters clusters = createNiceMock(Clusters.class);
 
     List<HostRoleCommand> hostRoleCommands = new LinkedList<HostRoleCommand>();
     hostRoleCommands.add(hostRoleCommand);
 
+    Collection<Stage> stages = new HashSet<Stage>();
+    stages.add(stage);
+
     org.apache.ambari.server.actionmanager.Request requestMock =
         createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
     expect(requestMock.getCommands()).andReturn(hostRoleCommands).anyTimes();
+    expect(requestMock.getStages()).andReturn(stages).anyTimes();
     expect(requestMock.getRequestContext()).andReturn("this is a context").anyTimes();
     expect(requestMock.getClusterName()).andReturn(null).anyTimes();
     expect(requestMock.getRequestId()).andReturn(100L).anyTimes();
 
+    expect(stage.getOrderedHostRoleCommands()).andReturn(hostRoleCommands).anyTimes();
+
     Capture<Collection<Long>> requestIdsCapture = new Capture<Collection<Long>>();
 
     // set expectations
@@ -1198,10 +1287,10 @@ public class RequestResourceProviderTest {
     expect(clusters.getCluster(anyObject(String.class))).andReturn(null).anyTimes();
     expect(actionManager.getRequests(capture(requestIdsCapture))).andReturn(Collections.singletonList(requestMock));
     expect(hostRoleCommand.getRequestId()).andReturn(100L).anyTimes();
-    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS);
+    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS).anyTimes();
 
     // replay
-    replay(managementController, actionManager, hostRoleCommand, clusters, requestMock);
+    replay(managementController, actionManager, hostRoleCommand, clusters, requestMock, stage);
 
     ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
         type,
@@ -1228,7 +1317,7 @@ public class RequestResourceProviderTest {
     }
 
     // verify
-    verify(managementController, actionManager, hostRoleCommand, clusters);
+    verify(managementController, actionManager, hostRoleCommand, clusters, stage);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/b11fe881/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/StageResourceProviderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/StageResourceProviderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/StageResourceProviderTest.java
index 87398a5..c97de8a 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/StageResourceProviderTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/StageResourceProviderTest.java
@@ -178,144 +178,6 @@ public class StageResourceProviderTest {
     verify(dao, clusters, cluster);
   }
 
-  @Test
-  public void testCalculateTaskStatusCounts() {
-
-    Collection<HostRoleStatus> hostRoleStatuses = new LinkedList<HostRoleStatus>();
-
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.QUEUED);
-    hostRoleStatuses.add(HostRoleStatus.HOLDING);
-    hostRoleStatuses.add(HostRoleStatus.HOLDING_FAILED);
-    hostRoleStatuses.add(HostRoleStatus.HOLDING_TIMEDOUT);
-    hostRoleStatuses.add(HostRoleStatus.IN_PROGRESS);
-    hostRoleStatuses.add(HostRoleStatus.IN_PROGRESS);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.FAILED);
-    hostRoleStatuses.add(HostRoleStatus.TIMEDOUT);
-    hostRoleStatuses.add(HostRoleStatus.ABORTED);
-
-    Map<HostRoleStatus, Integer> counts = StageResourceProvider.calculateTaskStatusCounts(hostRoleStatuses);
-
-    assertEquals(1L, (long) counts.get(HostRoleStatus.PENDING));
-    assertEquals(1L, (long) counts.get(HostRoleStatus.QUEUED));
-    assertEquals(1L, (long) counts.get(HostRoleStatus.HOLDING));
-    assertEquals(1L, (long) counts.get(HostRoleStatus.HOLDING_FAILED));
-    assertEquals(1L, (long) counts.get(HostRoleStatus.HOLDING_TIMEDOUT));
-    assertEquals(5L, (long) counts.get(HostRoleStatus.IN_PROGRESS));
-    assertEquals(7L, (long) counts.get(HostRoleStatus.COMPLETED));
-    assertEquals(1L, (long) counts.get(HostRoleStatus.FAILED));
-    assertEquals(1L, (long) counts.get(HostRoleStatus.TIMEDOUT));
-    assertEquals(1L, (long) counts.get(HostRoleStatus.ABORTED));
-  }
-
-  @Test
-  public void testCalculateProgressPercent() {
-
-    Collection<HostRoleStatus> hostRoleStatuses = new LinkedList<HostRoleStatus>();
-
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.QUEUED);
-    hostRoleStatuses.add(HostRoleStatus.HOLDING);
-    hostRoleStatuses.add(HostRoleStatus.IN_PROGRESS);
-    hostRoleStatuses.add(HostRoleStatus.IN_PROGRESS);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.FAILED);
-    hostRoleStatuses.add(HostRoleStatus.TIMEDOUT);
-    hostRoleStatuses.add(HostRoleStatus.ABORTED);
-
-    Map<HostRoleStatus, Integer> counts = StageResourceProvider.calculateTaskStatusCounts(hostRoleStatuses);
-
-    Double percent = StageResourceProvider.calculateProgressPercent(counts, counts.size());
-
-    assertEquals(Double.valueOf(64.9), percent);
-  }
-
-  @Test
-  public void testCalculateSummaryStatus() {
-
-    Collection<HostRoleStatus> hostRoleStatuses = new LinkedList<HostRoleStatus>();
-
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-
-    Map<HostRoleStatus, Integer> counts = StageResourceProvider.calculateTaskStatusCounts(hostRoleStatuses);
-
-    assertEquals(HostRoleStatus.PENDING, StageResourceProvider.calculateSummaryStatus(counts, hostRoleStatuses.size(), true));
-
-    hostRoleStatuses = new LinkedList<HostRoleStatus>();
-
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.FAILED);
-    hostRoleStatuses.add(HostRoleStatus.ABORTED);
-    hostRoleStatuses.add(HostRoleStatus.ABORTED);
-    hostRoleStatuses.add(HostRoleStatus.ABORTED);
-    hostRoleStatuses.add(HostRoleStatus.ABORTED);
-
-    counts = StageResourceProvider.calculateTaskStatusCounts(hostRoleStatuses);
-
-    assertEquals(HostRoleStatus.FAILED, StageResourceProvider.calculateSummaryStatus(counts, hostRoleStatuses.size(), true));
-
-    hostRoleStatuses = new LinkedList<HostRoleStatus>();
-
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.FAILED);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-
-    counts = StageResourceProvider.calculateTaskStatusCounts(hostRoleStatuses);
-
-    assertEquals(HostRoleStatus.COMPLETED, StageResourceProvider.calculateSummaryStatus(counts, hostRoleStatuses.size(), false));
-
-
-    hostRoleStatuses = new LinkedList<HostRoleStatus>();
-
-    counts = StageResourceProvider.calculateTaskStatusCounts(hostRoleStatuses);
-
-    assertEquals(HostRoleStatus.PENDING, StageResourceProvider.calculateSummaryStatus(counts, hostRoleStatuses.size(), false));
-
-
-    hostRoleStatuses = new LinkedList<HostRoleStatus>();
-
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.IN_PROGRESS);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-
-    counts = StageResourceProvider.calculateTaskStatusCounts(hostRoleStatuses);
-
-    assertEquals(HostRoleStatus.IN_PROGRESS, StageResourceProvider.calculateSummaryStatus(counts, hostRoleStatuses.size(), false));
-
-
-    hostRoleStatuses = new LinkedList<HostRoleStatus>();
-
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.COMPLETED);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-    hostRoleStatuses.add(HostRoleStatus.PENDING);
-
-    counts = StageResourceProvider.calculateTaskStatusCounts(hostRoleStatuses);
-
-    // this used to be PENDING; Changing behavior so that IN_PROGRESS is returned if any resource used in the calculation
-    // is past PENDING; AMBARI-8811
-    assertEquals(HostRoleStatus.IN_PROGRESS, StageResourceProvider.calculateSummaryStatus(counts, hostRoleStatuses.size(), false));
-  }
-
-
   private List<StageEntity> getStageEntities() {
     StageEntity stage = new StageEntity();