Browse Source

AMBARI-8643 - Expose Stage and Upgrade Item update status API. (tbeerbower)

tbeerbower 10 năm trước cách đây
mục cha
commit
6247e8bdeb
21 tập tin đã thay đổi với 269 bổ sung315 xóa
  1. 20 15
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java
  2. 29 29
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionScheduler.java
  3. 24 2
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleStatus.java
  4. 1 1
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Stage.java
  5. 0 97
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/StageStatus.java
  6. 58 13
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StageResourceProvider.java
  7. 3 3
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeItemResourceProvider.java
  8. 21 0
      ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java
  9. 1 2
      ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ManualStageAction.java
  10. 3 0
      ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog200.java
  11. 1 0
      ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
  12. 1 0
      ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
  13. 1 0
      ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
  14. 1 0
      ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql
  15. 1 1
      ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql
  16. 20 0
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/HostRoleStatusTest.java
  17. 0 132
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/StageStatusTest.java
  18. 5 2
      ambari-server/src/test/java/org/apache/ambari/server/controller/internal/RequestResourceProviderTest.java
  19. 30 18
      ambari-server/src/test/java/org/apache/ambari/server/controller/internal/StageResourceProviderTest.java
  20. 36 0
      ambari-server/src/test/java/org/apache/ambari/server/serveraction/ServerActionExecutorTest.java
  21. 13 0
      ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog200Test.java

+ 20 - 15
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java

@@ -179,7 +179,7 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
     List<HostRoleCommandEntity> commands =
         hostRoleCommandDAO.findByHostRole(host, requestId, stageId, role);
     for (HostRoleCommandEntity command : commands) {
-      command.setStatus(HostRoleStatus.TIMEDOUT);
+      command.setStatus(command.isRetryAllowed() ? HostRoleStatus.HOLDING_TIMEDOUT : HostRoleStatus.TIMEDOUT);
       command.setEndTime(now);
     }
     hostRoleCommandDAO.mergeAll(commands);
@@ -194,7 +194,8 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
     List<Stage> stages = new ArrayList<Stage>();
     List<HostRoleStatus> statuses =
         Arrays.asList(HostRoleStatus.QUEUED, HostRoleStatus.IN_PROGRESS,
-          HostRoleStatus.PENDING);
+          HostRoleStatus.PENDING, HostRoleStatus.HOLDING,
+          HostRoleStatus.HOLDING_FAILED, HostRoleStatus.HOLDING_TIMEDOUT);
     for (StageEntity stageEntity : stageDAO.findByCommandStatuses(statuses)) {
       stages.add(stageFactory.createExisting(stageEntity));
     }
@@ -207,7 +208,7 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
 
     RequestEntity requestEntity = request.constructNewPersistenceEntity();
 
-    Long clusterId = Long.valueOf(-1L);
+    Long clusterId = -1L;
     ClusterEntity clusterEntity = clusterDAO.findById(request.getClusterId());
     if (clusterEntity != null) {
       clusterId = clusterEntity.getClusterId();
@@ -228,11 +229,9 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
       stageDAO.create(stageEntity);
 
       List<HostRoleCommand> orderedHostRoleCommands = stage.getOrderedHostRoleCommands();
-      List<HostRoleCommandEntity> hostRoleCommandEntities = new ArrayList<HostRoleCommandEntity>();
 
       for (HostRoleCommand hostRoleCommand : orderedHostRoleCommands) {
         HostRoleCommandEntity hostRoleCommandEntity = hostRoleCommand.constructNewPersistenceEntity();
-        hostRoleCommandEntities.add(hostRoleCommandEntity);
         hostRoleCommandEntity.setStage(stageEntity);
 
         HostEntity hostEntity = hostDAO.findByName(hostRoleCommandEntity.getHostName());
@@ -354,7 +353,12 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
       if (commandEntity.getStatus() != HostRoleStatus.ABORTED) {
         // We don't want to overwrite statuses for ABORTED tasks with
         // statuses that have been received from the agent after aborting task
-        commandEntity.setStatus(HostRoleStatus.valueOf(report.getStatus()));
+        HostRoleStatus status = HostRoleStatus.valueOf(report.getStatus());
+        // if FAILED and marked for holding then set status = HOLDING_FAILED
+        if (status == HostRoleStatus.FAILED && commandEntity.isRetryAllowed()) {
+          status = HostRoleStatus.HOLDING_FAILED;
+        }
+        commandEntity.setStatus(status);
       } else {
         abortedCommandUpdates.add(commandEntity.getTaskId());
       }
@@ -399,7 +403,12 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
     List<HostRoleCommandEntity> commands = hostRoleCommandDAO.findByHostRole(
       hostname, requestId, stageId, role);
     for (HostRoleCommandEntity command : commands) {
-      command.setStatus(HostRoleStatus.valueOf(report.getStatus()));
+      HostRoleStatus status = HostRoleStatus.valueOf(report.getStatus());
+      // if FAILED and marked for holding then set status = HOLDING_FAILED
+      if (status == HostRoleStatus.FAILED && command.isRetryAllowed()) {
+        status = HostRoleStatus.HOLDING_FAILED;
+      }
+      command.setStatus(status);
       command.setStdOut(report.getStdOut().getBytes());
       command.setStdError(report.getStdErr().getBytes());
       command.setStructuredOut(report.getStructuredOut() == null ? null :
@@ -476,10 +485,7 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
 
   @Override
   public List<HostRoleCommand> getRequestTasks(long requestId) {
-    List<HostRoleCommand> tasks = new ArrayList<HostRoleCommand>();
-    return getTasks(
-        hostRoleCommandDAO.findTaskIdsByRequest(requestId)
-    );
+    return getTasks(hostRoleCommandDAO.findTaskIdsByRequest(requestId));
   }
 
   @Override
@@ -500,10 +506,8 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
 
     } else if (requestIds.isEmpty()) {
       return getTasks(taskIds);
-    } else if (taskIds.isEmpty()) {
-      return getAllTasksByRequestIds(requestIds);
     } else {
-      return Collections.emptyList();
+      return getAllTasksByRequestIds(requestIds);
     }
   }
 
@@ -585,7 +589,8 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
     Set<HostRoleStatus> statuses = new HashSet<HostRoleStatus>();
     if (status == RequestStatus.IN_PROGRESS) {
       statuses.addAll(Arrays.asList(HostRoleStatus.PENDING,
-          HostRoleStatus.IN_PROGRESS, HostRoleStatus.QUEUED));
+          HostRoleStatus.IN_PROGRESS, HostRoleStatus.QUEUED,
+          HostRoleStatus.HOLDING, HostRoleStatus.HOLDING_FAILED, HostRoleStatus.HOLDING_TIMEDOUT));
     } else if (status == RequestStatus.COMPLETED) {
       match = false;
       checkAllTasks = true;

+ 29 - 29
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionScheduler.java

@@ -39,7 +39,6 @@ import org.apache.ambari.server.ServiceComponentNotFoundException;
 import org.apache.ambari.server.agent.ActionQueue;
 import org.apache.ambari.server.agent.AgentCommand.AgentCommandType;
 import org.apache.ambari.server.agent.CancelCommand;
-import org.apache.ambari.server.agent.CommandReport;
 import org.apache.ambari.server.agent.ExecutionCommand;
 import org.apache.ambari.server.configuration.Configuration;
 import org.apache.ambari.server.controller.HostsMap;
@@ -266,8 +265,11 @@ class ActionScheduler implements Runnable {
         Map<String, RoleStats> roleStats = processInProgressStage(s, commandsToSchedule);
         // Check if stage is failed
         boolean failed = false;
-        for (String role : roleStats.keySet()) {
-          RoleStats stats = roleStats.get(role);
+        for (Map.Entry<String, RoleStats>entry : roleStats.entrySet()) {
+
+          String    role  = entry.getKey();
+          RoleStats stats = entry.getValue();
+
           if (LOG.isDebugEnabled()) {
             LOG.debug("Stats for role:" + role + ", stats=" + stats);
           }
@@ -348,7 +350,7 @@ class ActionScheduler implements Runnable {
         LOG.debug("==> Adding {} tasks to queue...", commandsToUpdate.size());
         for (ExecutionCommand cmd : commandsToUpdate) {
           // Do not queue up server actions; however if we encounter one, wake up the ServerActionExecutor
-          if (Role.AMBARI_SERVER_ACTION.toString().equals(cmd.getRole())) {
+          if (Role.AMBARI_SERVER_ACTION.name().equals(cmd.getRole())) {
             serverActionExecutor.awake();
           } else {
             actionQueue.enqueue(cmd.getHostname(), cmd);
@@ -457,11 +459,11 @@ class ActionScheduler implements Runnable {
   }
 
   /**
-   * @return Stats for the roles in the stage. It is used to determine whether stage
-   * has succeeded or failed.
-   * Side effects:
    * This method processes command timeouts and retry attempts, and
    * adds new (pending) execution commands to commandsToSchedule list.
+   *
+   * @return the stats for the roles in the stage which are used to determine
+   * whether stage has succeeded or failed
    */
   private Map<String, RoleStats> processInProgressStage(Stage s,
       List<ExecutionCommand> commandsToSchedule) throws AmbariException {
@@ -674,7 +676,7 @@ class ActionScheduler implements Runnable {
     for (Role r : hostCountsForRoles.keySet()) {
       RoleStats stats = new RoleStats(hostCountsForRoles.get(r),
           s.getSuccessFactor(r));
-      roleStats.put(r.toString(), stats);
+      roleStats.put(r.name(), stats);
     }
     return roleStats;
   }
@@ -856,7 +858,7 @@ class ActionScheduler implements Runnable {
                   fsmObject.getCluster(clusterName).getClusterId() : null;
           ActionFinalReportReceivedEvent event = new ActionFinalReportReceivedEvent(
                   clusterId, hostRoleCommand.getHostName(), null,
-                  hostRoleCommand.getRole().toString());
+                  hostRoleCommand.getRole().name());
           ambariEventPublisher.publish(event);
         } catch (AmbariException e) {
           LOG.error(String.format("Can not get cluster %s", clusterName), e);
@@ -888,6 +890,11 @@ class ActionScheduler implements Runnable {
     case IN_PROGRESS:
       rs.numInProgress++;
       break;
+    case HOLDING:
+    case HOLDING_FAILED:
+    case HOLDING_TIMEDOUT:
+      rs.numHolding++;
+      break;
     default:
       LOG.error("Unknown status " + status.name());
     }
@@ -910,6 +917,7 @@ class ActionScheduler implements Runnable {
     int numTimedOut = 0;
     int numPending = 0;
     int numAborted = 0;
+    int numHolding = 0;
     final int totalHosts;
     final float successFactor;
 
@@ -923,15 +931,11 @@ class ActionScheduler implements Runnable {
      */
     boolean isSuccessFactorMet() {
       int minSuccessNeeded = (int) Math.ceil(successFactor * totalHosts);
-      if (minSuccessNeeded <= numSucceeded) {
-        return true;
-      } else {
-        return false;
-      }
+      return minSuccessNeeded <= numSucceeded;
     }
 
     private boolean isRoleInProgress() {
-      return (numPending+numQueued+numInProgress > 0);
+      return numPending + numQueued + numInProgress + numHolding > 0;
     }
 
     /**
@@ -939,24 +943,20 @@ class ActionScheduler implements Runnable {
      * not met.
      */
     boolean isRoleFailed() {
-      if (isRoleInProgress() || isSuccessFactorMet()) {
-        return false;
-      } else {
-        return true;
-      }
+      return !(isRoleInProgress() || isSuccessFactorMet());
     }
 
     public String toString() {
       StringBuilder builder = new StringBuilder();
-      builder.append("numQueued="+numQueued);
-      builder.append(", numInProgress="+numInProgress);
-      builder.append(", numSucceeded="+numSucceeded);
-      builder.append(", numFailed="+numFailed);
-      builder.append(", numTimedOut="+numTimedOut);
-      builder.append(", numPending="+numPending);
-      builder.append(", numAborted="+numAborted);
-      builder.append(", totalHosts="+totalHosts);
-      builder.append(", successFactor="+successFactor);
+      builder.append("numQueued=").append(numQueued);
+      builder.append(", numInProgress=").append(numInProgress);
+      builder.append(", numSucceeded=").append(numSucceeded);
+      builder.append(", numFailed=").append(numFailed);
+      builder.append(", numTimedOut=").append(numTimedOut);
+      builder.append(", numPending=").append(numPending);
+      builder.append(", numAborted=").append(numAborted);
+      builder.append(", totalHosts=").append(totalHosts);
+      builder.append(", successFactor=").append(successFactor);
       return builder.toString();
     }
   }

+ 24 - 2
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleStatus.java

@@ -34,6 +34,10 @@ public enum HostRoleStatus {
    * Host reported it is working, received an IN_PROGRESS command status from host.
    */
   IN_PROGRESS,
+  /**
+   * Task is holding, waiting for command to proceed to completion.
+   */
+  HOLDING,
   /**
    * Host reported success
    */
@@ -42,10 +46,18 @@ public enum HostRoleStatus {
    * Failed
    */
   FAILED,
+  /**
+   * Task is holding after a failure, waiting for command to skip or retry.
+   */
+  HOLDING_FAILED,
   /**
    * Host did not respond in time
    */
   TIMEDOUT,
+  /**
+   * Task is holding after a time-out, waiting for command to skip or retry.
+   */
+  HOLDING_TIMEDOUT,
   /**
    * Operation was abandoned
    */
@@ -53,6 +65,7 @@ public enum HostRoleStatus {
 
   private static List<HostRoleStatus> COMPLETED_STATES = Arrays.asList(FAILED, TIMEDOUT, ABORTED, COMPLETED);
   private static List<HostRoleStatus> FAILED_STATES = Arrays.asList(FAILED, TIMEDOUT, ABORTED);
+  private static List<HostRoleStatus> HOLDING_STATES = Arrays.asList(HOLDING, HOLDING_FAILED, HOLDING_TIMEDOUT);
 
 
   /**
@@ -76,6 +89,17 @@ public enum HostRoleStatus {
     return COMPLETED_STATES.contains(this);
   }
 
+  /**
+   * Indicates whether or not this is a holding state.
+   * Holding means that the associated task is waiting for
+   * a command to transition to a completion state.
+   *
+   * @return true if this is a holding state.
+   */
+  public boolean isHoldingState() {
+    return HOLDING_STATES.contains(this);
+  }
+
   /**
    *
    * @return list of completed states
@@ -91,6 +115,4 @@ public enum HostRoleStatus {
   public static List<HostRoleStatus> getFailedStates() {
     return Collections.unmodifiableList(FAILED_STATES);
   }
-
-
 }

+ 1 - 1
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Stage.java

@@ -342,7 +342,7 @@ public class Stage {
    * @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 hostName      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

+ 0 - 97
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/StageStatus.java

@@ -1,97 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ambari.server.actionmanager;
-
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Stage status enum.
- */
-public enum StageStatus {
-
-  /**
-   * Stage contains tasks not yet queued for a host.
-   */
-  PENDING,
-
-  /**
-   * Stage contains tasks that are reported to be in progress.
-   */
-  IN_PROGRESS,
-
-  /**
-   * Stage is holding, waiting for command to proceed to next stage.
-   */
-  HOLDING,
-
-  /**
-   * All tasks for this stage have completed.
-   */
-  COMPLETED,
-
-  /**
-   * At least one task for this stage has reported a failure.
-   */
-  FAILED,
-
-  /**
-   * Stage is holding after a failure, waiting for command to proceed to next stage.
-   */
-  HOLDING_FAILED,
-
-  /**
-   * At least one task for this stage has timed out.
-   */
-  TIMEDOUT,
-
-  /**
-   * Stage is holding after a time-out, waiting for command to proceed to next stage.
-   */
-  HOLDING_TIMEDOUT,
-
-  /**
-   * Operation was abandoned.
-   */
-  ABORTED;
-
-  /**
-   * Mapping of valid status transitions that that are driven by manual input.
-   */
-  private static Map<StageStatus, EnumSet<StageStatus>> manualTransitionMap = new HashMap<StageStatus, EnumSet<StageStatus>>();
-
-  static {
-    manualTransitionMap.put(HOLDING, EnumSet.of(COMPLETED));
-    manualTransitionMap.put(HOLDING_FAILED, EnumSet.of(IN_PROGRESS, FAILED));
-    manualTransitionMap.put(HOLDING_TIMEDOUT, EnumSet.of(IN_PROGRESS, TIMEDOUT));
-  }
-
-  /**
-   * Determine whether or not it is valid to transition from this stage status to the given status.
-   *
-   * @param status  the stage status being transitioned to
-   *
-   * @return true if it is valid to transition to the given stage status
-   */
-  public boolean isValidManualTransition(StageStatus status) {
-    EnumSet<StageStatus> stageStatusSet = manualTransitionMap.get(this);
-    return stageStatusSet != null && stageStatusSet.contains(status);
-  }
-}

+ 58 - 13
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StageResourceProvider.java

@@ -19,7 +19,6 @@ package org.apache.ambari.server.controller.internal;
 
 import org.apache.ambari.server.StaticallyInject;
 import org.apache.ambari.server.actionmanager.HostRoleStatus;
-import org.apache.ambari.server.actionmanager.StageStatus;
 import org.apache.ambari.server.controller.spi.ExtendedResourceProvider;
 import org.apache.ambari.server.controller.spi.NoSuchParentResourceException;
 import org.apache.ambari.server.controller.spi.NoSuchResourceException;
@@ -33,6 +32,7 @@ import org.apache.ambari.server.controller.spi.SystemException;
 import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
 import org.apache.ambari.server.controller.utilities.PredicateBuilder;
 import org.apache.ambari.server.controller.utilities.PropertyHelper;
+import org.apache.ambari.server.orm.dao.HostRoleCommandDAO;
 import org.apache.ambari.server.orm.dao.StageDAO;
 import org.apache.ambari.server.orm.entities.HostRoleCommandEntity;
 import org.apache.ambari.server.orm.entities.StageEntity;
@@ -42,6 +42,7 @@ import org.apache.ambari.server.state.Clusters;
 import javax.inject.Inject;
 import javax.inject.Provider;
 import java.util.Collection;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -62,6 +63,12 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
   @Inject
   private static StageDAO dao = null;
 
+  /**
+   * Used for querying task resources.
+   */
+  @Inject
+  private static HostRoleCommandDAO hostRoleCommandDAO = null;
+
   /**
    * Used to get cluster information.
    */
@@ -118,6 +125,17 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
     KEY_PROPERTY_IDS.put(Resource.Type.Request, STAGE_REQUEST_ID);
   }
 
+  /**
+   * Mapping of valid status transitions that that are driven by manual input.
+   */
+  private static Map<HostRoleStatus, EnumSet<HostRoleStatus>> manualTransitionMap = new HashMap<HostRoleStatus, EnumSet<HostRoleStatus>>();
+
+  static {
+    manualTransitionMap.put(HostRoleStatus.HOLDING, EnumSet.of(HostRoleStatus.COMPLETED));
+    manualTransitionMap.put(HostRoleStatus.HOLDING_FAILED, EnumSet.of(HostRoleStatus.PENDING, HostRoleStatus.FAILED));
+    manualTransitionMap.put(HostRoleStatus.HOLDING_TIMEDOUT, EnumSet.of(HostRoleStatus.PENDING, HostRoleStatus.TIMEDOUT));
+  }
+
 
   // ----- Constructors ------------------------------------------------------
 
@@ -162,7 +180,7 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
 
         String stageStatus = (String) updateProperties.get(STAGE_STATUS);
         if (stageStatus != null) {
-          StageStatus desiredStatus = StageStatus.valueOf(stageStatus);
+          HostRoleStatus desiredStatus = HostRoleStatus.valueOf(stageStatus);
           updateStageStatus(entity, desiredStatus);
         }
       }
@@ -217,7 +235,7 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
    * @param stageId        the stage id
    * @param desiredStatus  the desired stage status
    */
-  public static void updateStageStatus(long stageId, StageStatus desiredStatus) {
+  public static void updateStageStatus(long stageId, HostRoleStatus desiredStatus) {
     Predicate predicate =
         new PredicateBuilder().property(STAGE_STAGE_ID).equals(stageId).toPredicate();
 
@@ -239,18 +257,30 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
    * @throws java.lang.IllegalArgumentException if the transition to the desired status is not a
    *         legal transition
    */
-  private static void updateStageStatus(StageEntity entity, StageStatus desiredStatus) {
+  private static void updateStageStatus(StageEntity entity, HostRoleStatus desiredStatus) {
     Collection<HostRoleCommandEntity> tasks = entity.getHostRoleCommands();
 
     Map<HostRoleStatus, Integer> taskStatusCounts = calculateTaskStatusCounts(tasks);
 
-    StageStatus currentStatus = calculateSummaryStatus(taskStatusCounts, tasks.size());
+    HostRoleStatus currentStatus = calculateSummaryStatus(taskStatusCounts, tasks.size());
 
-    if (!currentStatus.isValidManualTransition(desiredStatus)) {
+    if (!isValidManualTransition(currentStatus, desiredStatus)) {
       throw new IllegalArgumentException("Can not transition a stage from " +
           currentStatus + " to " + desiredStatus);
     }
-    // TODO : call ActionScheduler to release holding state
+
+    for (HostRoleCommandEntity hostRoleCommand : tasks) {
+      HostRoleStatus hostRoleStatus = hostRoleCommand.getStatus();
+      if (hostRoleStatus.equals(currentStatus)) {
+        hostRoleCommand.setStatus(desiredStatus);
+
+        if (desiredStatus == HostRoleStatus.PENDING) {
+          hostRoleCommand.setStartTime(-1L);
+        }
+
+        hostRoleCommandDAO.merge(hostRoleCommand);
+      }
+    }
   }
 
   /**
@@ -331,12 +361,15 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
    *
    * @return summary request status based on statuses of tasks in different states.
    */
-  private static StageStatus calculateSummaryStatus(Map<HostRoleStatus, Integer> counters, int totalTasks) {
-    return counters.get(HostRoleStatus.FAILED) > 0 ? StageStatus.FAILED :
-        counters.get(HostRoleStatus.ABORTED) > 0 ? StageStatus.ABORTED :
-        counters.get(HostRoleStatus.TIMEDOUT) > 0 ? StageStatus.TIMEDOUT :
-        counters.get(HostRoleStatus.IN_PROGRESS) > 0 ? StageStatus.IN_PROGRESS :
-        counters.get(HostRoleStatus.COMPLETED) == totalTasks ? StageStatus.COMPLETED : StageStatus.PENDING;
+  private static HostRoleStatus calculateSummaryStatus(Map<HostRoleStatus, Integer> counters, int totalTasks) {
+    return 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 ? HostRoleStatus.FAILED :
+        counters.get(HostRoleStatus.ABORTED) > 0 ? HostRoleStatus.ABORTED :
+        counters.get(HostRoleStatus.TIMEDOUT) > 0 ? HostRoleStatus.TIMEDOUT :
+        counters.get(HostRoleStatus.IN_PROGRESS) > 0 ? HostRoleStatus.IN_PROGRESS :
+        counters.get(HostRoleStatus.COMPLETED) == totalTasks ? HostRoleStatus.COMPLETED : HostRoleStatus.PENDING;
   }
 
   /**
@@ -386,4 +419,16 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
     }
     return clusters;
   }
+
+  /**
+   * Determine whether or not it is valid to transition from this stage status to the given status.
+   *
+   * @param status  the stage status being transitioned to
+   *
+   * @return true if it is valid to transition to the given stage status
+   */
+  private static boolean isValidManualTransition(HostRoleStatus status, HostRoleStatus desiredStatus) {
+    EnumSet<HostRoleStatus> stageStatusSet = manualTransitionMap.get(status);
+    return stageStatusSet != null && stageStatusSet.contains(desiredStatus);
+  }
 }

+ 3 - 3
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeItemResourceProvider.java

@@ -27,7 +27,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.ambari.server.StaticallyInject;
-import org.apache.ambari.server.actionmanager.StageStatus;
+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;
@@ -112,8 +112,8 @@ public class UpgradeItemResourceProvider extends ReadOnlyResourceProvider {
 
       if (stageStatus != null) {
 
-        StageStatus   desiredStatus = StageStatus.valueOf(stageStatus);
-        Set<Resource> resources     = getResources(PropertyHelper.getReadRequest(), predicate);
+        HostRoleStatus desiredStatus = HostRoleStatus.valueOf(stageStatus);
+        Set<Resource>  resources     = getResources(PropertyHelper.getReadRequest(), predicate);
 
         for (Resource resource : resources) {
           // Set the desired status on the underlying stage.

+ 21 - 0
ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java

@@ -131,6 +131,9 @@ public class HostRoleCommandEntity {
   @Column(name = "attempt_count", nullable = false)
   private Short attemptCount = 0;
 
+  @Column(name = "retry_allowed", nullable = false)
+  private Integer retryAllowed = Integer.valueOf(0);
+
   // This is really command type as well as name
   @Column(name = "role_command")
   @Enumerated(EnumType.STRING)
@@ -315,6 +318,24 @@ public class HostRoleCommandEntity {
     this.customCommandName = customCommandName;
   }
 
+  /**
+   * Determine whether this task should hold for retry when an error occurs.
+   *
+   * @return {@code true} if this task should hold for retry when an error occurs
+   */
+  public boolean isRetryAllowed() {
+    return retryAllowed != 0;
+  }
+
+  /**
+   * Sets whether this task should hold for retry when an error occurs.
+   *
+   * @param enabled  {@code true} if this task should hold for retry when an error occurs
+   */
+  public void setRetryAllowed(boolean enabled) {
+    this.retryAllowed = enabled ? 1 : 0;
+  }
+
   @Override
   public boolean equals(Object o) {
     if (this == o) return true;

+ 1 - 2
ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/ManualStageAction.java

@@ -34,7 +34,6 @@ public class ManualStageAction extends AbstractServerAction {
       ConcurrentMap<String, Object> requestSharedDataContext)
       throws AmbariException, InterruptedException {
     // TODO Auto-generated method stub
-    return createCommandReport(0, HostRoleStatus.COMPLETED, "{}", "", "");
+    return createCommandReport(0, HostRoleStatus.HOLDING, "{}", "", "");
   }
-
 }

+ 3 - 0
ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog200.java

@@ -183,6 +183,9 @@ public class UpgradeCatalog200 extends AbstractUpgradeCatalog {
     dbAccessor.addColumn("hostcomponentstate", new DBAccessor.DBColumnInfo("upgrade_state",
         String.class, 32, "NONE", false));
 
+    dbAccessor.addColumn("host_role_command", new DBAccessor.DBColumnInfo("retry_allowed",
+        Integer.class, 1, 0, false));
+
     // New tables
     columns.add(new DBAccessor.DBColumnInfo("id", Long.class, null, null, false));
     columns.add(new DBAccessor.DBColumnInfo("cluster_id", Long.class, null, null, false));

+ 1 - 0
ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql

@@ -200,6 +200,7 @@ CREATE TABLE execution_command (
 CREATE TABLE host_role_command (
   task_id BIGINT NOT NULL,
   attempt_count SMALLINT NOT NULL,
+  retry_allowed SMALLINT DEFAULT 0 NOT NULL,
   event LONGTEXT NOT NULL,
   exitcode INTEGER NOT NULL,
   host_name VARCHAR(255) NOT NULL,

+ 1 - 0
ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql

@@ -191,6 +191,7 @@ CREATE TABLE execution_command (
 CREATE TABLE host_role_command (
   task_id NUMBER(19) NOT NULL,
   attempt_count NUMBER(5) NOT NULL,
+  retry_allowed NUMBER(1) DEFAULT 0 NOT NULL,
   event CLOB NULL,
   exitcode NUMBER(10) NOT NULL,
   host_name VARCHAR2(255) NOT NULL,

+ 1 - 0
ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql

@@ -201,6 +201,7 @@ CREATE TABLE execution_command (
 CREATE TABLE host_role_command (
   task_id BIGINT NOT NULL,
   attempt_count SMALLINT NOT NULL,
+  retry_allowed SMALLINT DEFAULT 0 NOT NULL,
   event VARCHAR(32000) NOT NULL,
   exitcode INTEGER NOT NULL,
   host_name VARCHAR(255) NOT NULL,

+ 1 - 0
ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql

@@ -234,6 +234,7 @@ GRANT ALL PRIVILEGES ON TABLE ambari.execution_command TO :username;
 CREATE TABLE ambari.host_role_command (
   task_id BIGINT NOT NULL,
   attempt_count SMALLINT NOT NULL,
+  retry_allowed SMALLINT DEFAULT 0 NOT NULL,
   event VARCHAR(32000) NOT NULL,
   exitcode INTEGER NOT NULL,
   host_name VARCHAR(255) NOT NULL,

+ 1 - 1
ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql

@@ -50,7 +50,7 @@ CREATE TABLE users (user_id INTEGER, principal_id BIGINT NOT NULL, ldap_user INT
 CREATE TABLE groups (group_id INTEGER, principal_id BIGINT NOT NULL, group_name VARCHAR(255) NOT NULL, ldap_group INTEGER NOT NULL DEFAULT 0, PRIMARY KEY (group_id));
 CREATE TABLE members (member_id INTEGER, group_id INTEGER NOT NULL, user_id INTEGER NOT NULL, PRIMARY KEY (member_id));
 CREATE TABLE execution_command (command VARBINARY(8000), task_id BIGINT NOT NULL, PRIMARY KEY CLUSTERED (task_id));
-CREATE TABLE host_role_command (task_id BIGINT NOT NULL, attempt_count SMALLINT NOT NULL, event VARCHAR(MAX) NOT NULL, exitcode INTEGER NOT NULL, host_name VARCHAR(255) NOT NULL, last_attempt_time BIGINT NOT NULL, request_id BIGINT NOT NULL, role VARCHAR(255), stage_id BIGINT NOT NULL, start_time BIGINT NOT NULL, end_time BIGINT, status VARCHAR(255), std_error VARBINARY(max), std_out VARBINARY(max), output_log VARCHAR(255) NULL, error_log VARCHAR(255) NULL, structured_out VARBINARY(max), role_command VARCHAR(255), command_detail VARCHAR(255), custom_command_name VARCHAR(255), PRIMARY KEY CLUSTERED (task_id));
+CREATE TABLE host_role_command (task_id BIGINT NOT NULL, attempt_count SMALLINT NOT NULL, retry_allowed SMALLINT DEFAULT 0 NOT NULL, event VARCHAR(MAX) NOT NULL, exitcode INTEGER NOT NULL, host_name VARCHAR(255) NOT NULL, last_attempt_time BIGINT NOT NULL, request_id BIGINT NOT NULL, role VARCHAR(255), stage_id BIGINT NOT NULL, start_time BIGINT NOT NULL, end_time BIGINT, status VARCHAR(255), std_error VARBINARY(max), std_out VARBINARY(max), output_log VARCHAR(255) NULL, error_log VARCHAR(255) NULL, structured_out VARBINARY(max), role_command VARCHAR(255), command_detail VARCHAR(255), custom_command_name VARCHAR(255), PRIMARY KEY CLUSTERED (task_id));
 CREATE TABLE role_success_criteria (role VARCHAR(255) NOT NULL, request_id BIGINT NOT NULL, stage_id BIGINT NOT NULL, success_factor FLOAT NOT NULL, PRIMARY KEY CLUSTERED (role, request_id, stage_id));
 CREATE TABLE stage (stage_id BIGINT NOT NULL, request_id BIGINT NOT NULL, cluster_id BIGINT NOT NULL, log_info VARCHAR(255) NOT NULL, request_context VARCHAR(255), cluster_host_info VARBINARY(8000) NOT NULL, command_params VARBINARY(8000), host_params VARBINARY(8000), PRIMARY KEY CLUSTERED (stage_id, request_id));
 CREATE TABLE request (request_id BIGINT NOT NULL, cluster_id BIGINT, command_name VARCHAR(255), create_time BIGINT NOT NULL, end_time BIGINT NOT NULL, exclusive_execution BIT NOT NULL DEFAULT 0, inputs VARBINARY(8000), request_context VARCHAR(255), request_type VARCHAR(255), request_schedule_id BIGINT, start_time BIGINT NOT NULL, status VARCHAR(255), PRIMARY KEY CLUSTERED (request_id));

+ 20 - 0
ambari-server/src/test/java/org/apache/ambari/server/actionmanager/HostRoleStatusTest.java

@@ -34,6 +34,9 @@ public class HostRoleStatusTest {
     Assert.assertFalse(HostRoleStatus.PENDING.isFailedState());
     Assert.assertFalse(HostRoleStatus.QUEUED.isFailedState());
     Assert.assertTrue(HostRoleStatus.TIMEDOUT.isFailedState());
+    Assert.assertFalse(HostRoleStatus.HOLDING.isFailedState());
+    Assert.assertFalse(HostRoleStatus.HOLDING_FAILED.isFailedState());
+    Assert.assertFalse(HostRoleStatus.HOLDING_TIMEDOUT.isFailedState());
   }
 
   @Test
@@ -45,5 +48,22 @@ public class HostRoleStatusTest {
     Assert.assertFalse(HostRoleStatus.PENDING.isCompletedState());
     Assert.assertFalse(HostRoleStatus.QUEUED.isCompletedState());
     Assert.assertTrue(HostRoleStatus.TIMEDOUT.isCompletedState());
+    Assert.assertFalse(HostRoleStatus.HOLDING.isCompletedState());
+    Assert.assertFalse(HostRoleStatus.HOLDING_FAILED.isCompletedState());
+    Assert.assertFalse(HostRoleStatus.HOLDING_TIMEDOUT.isCompletedState());
+  }
+
+  @Test
+  public void testIsHoldingState() throws Exception {
+    Assert.assertFalse(HostRoleStatus.ABORTED.isHoldingState());
+    Assert.assertFalse(HostRoleStatus.COMPLETED.isHoldingState());
+    Assert.assertFalse(HostRoleStatus.FAILED.isHoldingState());
+    Assert.assertFalse(HostRoleStatus.IN_PROGRESS.isHoldingState());
+    Assert.assertFalse(HostRoleStatus.PENDING.isHoldingState());
+    Assert.assertFalse(HostRoleStatus.QUEUED.isHoldingState());
+    Assert.assertFalse(HostRoleStatus.TIMEDOUT.isHoldingState());
+    Assert.assertTrue(HostRoleStatus.HOLDING.isHoldingState());
+    Assert.assertTrue(HostRoleStatus.HOLDING_FAILED.isHoldingState());
+    Assert.assertTrue(HostRoleStatus.HOLDING_TIMEDOUT.isHoldingState());
   }
 }

+ 0 - 132
ambari-server/src/test/java/org/apache/ambari/server/actionmanager/StageStatusTest.java

@@ -1,132 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ambari.server.actionmanager;
-
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-/**
- * StageStatus tests.
- */
-public class StageStatusTest {
-
-  @Test
-  public void testIsValidManualTransition() throws Exception {
-
-    // PENDING
-    assertFalse(StageStatus.PENDING.isValidManualTransition(StageStatus.PENDING));
-    assertFalse(StageStatus.PENDING.isValidManualTransition(StageStatus.IN_PROGRESS));
-    assertFalse(StageStatus.PENDING.isValidManualTransition(StageStatus.HOLDING));
-    assertFalse(StageStatus.PENDING.isValidManualTransition(StageStatus.COMPLETED));
-    assertFalse(StageStatus.PENDING.isValidManualTransition(StageStatus.FAILED));
-    assertFalse(StageStatus.PENDING.isValidManualTransition(StageStatus.HOLDING_FAILED));
-    assertFalse(StageStatus.PENDING.isValidManualTransition(StageStatus.TIMEDOUT));
-    assertFalse(StageStatus.PENDING.isValidManualTransition(StageStatus.HOLDING_TIMEDOUT));
-    assertFalse(StageStatus.PENDING.isValidManualTransition(StageStatus.ABORTED));
-
-    // IN_PROGRESS
-    assertFalse(StageStatus.IN_PROGRESS.isValidManualTransition(StageStatus.PENDING));
-    assertFalse(StageStatus.IN_PROGRESS.isValidManualTransition(StageStatus.IN_PROGRESS));
-    assertFalse(StageStatus.IN_PROGRESS.isValidManualTransition(StageStatus.HOLDING));
-    assertFalse(StageStatus.IN_PROGRESS.isValidManualTransition(StageStatus.COMPLETED));
-    assertFalse(StageStatus.IN_PROGRESS.isValidManualTransition(StageStatus.FAILED));
-    assertFalse(StageStatus.IN_PROGRESS.isValidManualTransition(StageStatus.HOLDING_FAILED));
-    assertFalse(StageStatus.IN_PROGRESS.isValidManualTransition(StageStatus.TIMEDOUT));
-    assertFalse(StageStatus.IN_PROGRESS.isValidManualTransition(StageStatus.HOLDING_TIMEDOUT));
-    assertFalse(StageStatus.IN_PROGRESS.isValidManualTransition(StageStatus.ABORTED));
-
-    // HOLDING
-    assertFalse(StageStatus.HOLDING.isValidManualTransition(StageStatus.PENDING));
-    assertFalse(StageStatus.HOLDING.isValidManualTransition(StageStatus.IN_PROGRESS));
-    assertFalse(StageStatus.HOLDING.isValidManualTransition(StageStatus.HOLDING));
-    assertTrue(StageStatus.HOLDING.isValidManualTransition(StageStatus.COMPLETED));
-    assertFalse(StageStatus.HOLDING.isValidManualTransition(StageStatus.FAILED));
-    assertFalse(StageStatus.HOLDING.isValidManualTransition(StageStatus.HOLDING_FAILED));
-    assertFalse(StageStatus.HOLDING.isValidManualTransition(StageStatus.TIMEDOUT));
-    assertFalse(StageStatus.HOLDING.isValidManualTransition(StageStatus.HOLDING_TIMEDOUT));
-    assertFalse(StageStatus.HOLDING.isValidManualTransition(StageStatus.ABORTED));
-
-    // COMPLETED
-    assertFalse(StageStatus.COMPLETED.isValidManualTransition(StageStatus.PENDING));
-    assertFalse(StageStatus.COMPLETED.isValidManualTransition(StageStatus.IN_PROGRESS));
-    assertFalse(StageStatus.COMPLETED.isValidManualTransition(StageStatus.HOLDING));
-    assertFalse(StageStatus.COMPLETED.isValidManualTransition(StageStatus.COMPLETED));
-    assertFalse(StageStatus.COMPLETED.isValidManualTransition(StageStatus.FAILED));
-    assertFalse(StageStatus.COMPLETED.isValidManualTransition(StageStatus.HOLDING_FAILED));
-    assertFalse(StageStatus.COMPLETED.isValidManualTransition(StageStatus.TIMEDOUT));
-    assertFalse(StageStatus.COMPLETED.isValidManualTransition(StageStatus.HOLDING_TIMEDOUT));
-    assertFalse(StageStatus.COMPLETED.isValidManualTransition(StageStatus.ABORTED));
-
-    // FAILED
-    assertFalse(StageStatus.FAILED.isValidManualTransition(StageStatus.PENDING));
-    assertFalse(StageStatus.FAILED.isValidManualTransition(StageStatus.IN_PROGRESS));
-    assertFalse(StageStatus.FAILED.isValidManualTransition(StageStatus.HOLDING));
-    assertFalse(StageStatus.FAILED.isValidManualTransition(StageStatus.COMPLETED));
-    assertFalse(StageStatus.FAILED.isValidManualTransition(StageStatus.FAILED));
-    assertFalse(StageStatus.FAILED.isValidManualTransition(StageStatus.HOLDING_FAILED));
-    assertFalse(StageStatus.FAILED.isValidManualTransition(StageStatus.TIMEDOUT));
-    assertFalse(StageStatus.FAILED.isValidManualTransition(StageStatus.HOLDING_TIMEDOUT));
-    assertFalse(StageStatus.FAILED.isValidManualTransition(StageStatus.ABORTED));
-
-    // HOLDING_FAILED
-    assertFalse(StageStatus.HOLDING_FAILED.isValidManualTransition(StageStatus.PENDING));
-    assertTrue(StageStatus.HOLDING_FAILED.isValidManualTransition(StageStatus.IN_PROGRESS));
-    assertFalse(StageStatus.HOLDING_FAILED.isValidManualTransition(StageStatus.HOLDING));
-    assertFalse(StageStatus.HOLDING_FAILED.isValidManualTransition(StageStatus.COMPLETED));
-    assertTrue(StageStatus.HOLDING_FAILED.isValidManualTransition(StageStatus.FAILED));
-    assertFalse(StageStatus.HOLDING_FAILED.isValidManualTransition(StageStatus.HOLDING_FAILED));
-    assertFalse(StageStatus.HOLDING_FAILED.isValidManualTransition(StageStatus.TIMEDOUT));
-    assertFalse(StageStatus.HOLDING_FAILED.isValidManualTransition(StageStatus.HOLDING_TIMEDOUT));
-    assertFalse(StageStatus.HOLDING_FAILED.isValidManualTransition(StageStatus.ABORTED));
-
-    // TIMEDOUT
-    assertFalse(StageStatus.TIMEDOUT.isValidManualTransition(StageStatus.PENDING));
-    assertFalse(StageStatus.TIMEDOUT.isValidManualTransition(StageStatus.IN_PROGRESS));
-    assertFalse(StageStatus.TIMEDOUT.isValidManualTransition(StageStatus.HOLDING));
-    assertFalse(StageStatus.TIMEDOUT.isValidManualTransition(StageStatus.COMPLETED));
-    assertFalse(StageStatus.TIMEDOUT.isValidManualTransition(StageStatus.FAILED));
-    assertFalse(StageStatus.TIMEDOUT.isValidManualTransition(StageStatus.HOLDING_FAILED));
-    assertFalse(StageStatus.TIMEDOUT.isValidManualTransition(StageStatus.TIMEDOUT));
-    assertFalse(StageStatus.TIMEDOUT.isValidManualTransition(StageStatus.HOLDING_TIMEDOUT));
-    assertFalse(StageStatus.TIMEDOUT.isValidManualTransition(StageStatus.ABORTED));
-
-    // HOLDING_TIMEDOUT
-    assertFalse(StageStatus.HOLDING_TIMEDOUT.isValidManualTransition(StageStatus.PENDING));
-    assertTrue(StageStatus.HOLDING_TIMEDOUT.isValidManualTransition(StageStatus.IN_PROGRESS));
-    assertFalse(StageStatus.HOLDING_TIMEDOUT.isValidManualTransition(StageStatus.HOLDING));
-    assertFalse(StageStatus.HOLDING_TIMEDOUT.isValidManualTransition(StageStatus.COMPLETED));
-    assertFalse(StageStatus.HOLDING_TIMEDOUT.isValidManualTransition(StageStatus.FAILED));
-    assertFalse(StageStatus.HOLDING_TIMEDOUT.isValidManualTransition(StageStatus.HOLDING_FAILED));
-    assertTrue(StageStatus.HOLDING_TIMEDOUT.isValidManualTransition(StageStatus.TIMEDOUT));
-    assertFalse(StageStatus.HOLDING_TIMEDOUT.isValidManualTransition(StageStatus.HOLDING_TIMEDOUT));
-    assertFalse(StageStatus.HOLDING_TIMEDOUT.isValidManualTransition(StageStatus.ABORTED));
-
-    // ABORTED
-    assertFalse(StageStatus.ABORTED.isValidManualTransition(StageStatus.PENDING));
-    assertFalse(StageStatus.ABORTED.isValidManualTransition(StageStatus.IN_PROGRESS));
-    assertFalse(StageStatus.ABORTED.isValidManualTransition(StageStatus.HOLDING));
-    assertFalse(StageStatus.ABORTED.isValidManualTransition(StageStatus.COMPLETED));
-    assertFalse(StageStatus.ABORTED.isValidManualTransition(StageStatus.FAILED));
-    assertFalse(StageStatus.ABORTED.isValidManualTransition(StageStatus.HOLDING_FAILED));
-    assertFalse(StageStatus.ABORTED.isValidManualTransition(StageStatus.TIMEDOUT));
-    assertFalse(StageStatus.ABORTED.isValidManualTransition(StageStatus.HOLDING_TIMEDOUT));
-    assertFalse(StageStatus.ABORTED.isValidManualTransition(StageStatus.ABORTED));
-  }
-}

+ 5 - 2
ambari-server/src/test/java/org/apache/ambari/server/controller/internal/RequestResourceProviderTest.java

@@ -842,8 +842,11 @@ public class RequestResourceProviderTest {
       replay(hostRoleCommand);
       request = PropertyHelper.getUpdateRequest(properties, null);
       if (status == HostRoleStatus.IN_PROGRESS ||
-              status == HostRoleStatus.PENDING ||
-              status == HostRoleStatus.QUEUED) { // the only valid cases
+          status == HostRoleStatus.PENDING ||
+          status == HostRoleStatus.HOLDING ||
+          status == HostRoleStatus.HOLDING_FAILED ||
+          status == HostRoleStatus.HOLDING_TIMEDOUT ||
+          status == HostRoleStatus.QUEUED) { // the only valid cases
         provider.updateResources(request, predicate);
       } else {  // In other cases, should error out
         try {

+ 30 - 18
ambari-server/src/test/java/org/apache/ambari/server/controller/internal/StageResourceProviderTest.java

@@ -34,17 +34,19 @@ import org.apache.ambari.server.orm.entities.HostRoleCommandEntity;
 import org.apache.ambari.server.orm.entities.StageEntity;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
-import org.easymock.EasyMock;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
+import static org.easymock.EasyMock.anyLong;
 import static org.easymock.EasyMock.createNiceMock;
 import static org.easymock.EasyMock.createStrictMock;
 import static org.easymock.EasyMock.expect;
@@ -55,11 +57,15 @@ import static org.junit.Assert.*;
 public class StageResourceProviderTest {
 
   private StageDAO dao = null;
+  private Clusters clusters = null;
+  private Cluster cluster = null;
   private Injector injector;
 
   @Before
   public void before() {
     dao = createStrictMock(StageDAO.class);
+    clusters = createStrictMock(Clusters.class);
+    cluster = createStrictMock(Cluster.class);
 
     // create an injector which will inject the mocks
     injector = Guice.createInjector(Modules.override(
@@ -86,15 +92,17 @@ public class StageResourceProviderTest {
   public void testUpdateResources() throws Exception {
     StageResourceProvider provider = new StageResourceProvider();
 
-    // TODO : update test to account for stage status update
-//    Request request = createNiceMock(Request.class);
-//    Predicate predicate = createNiceMock(Predicate.class);
-//    try {
-//      provider.updateResources(request, predicate);
-//      fail("Expected UnsupportedOperationException");
-//    } catch (UnsupportedOperationException e) {
-//      // expected
-//    }
+    Request request = createNiceMock(Request.class);
+    Predicate predicate = createNiceMock(Predicate.class);
+
+    expect(clusters.getClusterById(anyLong())).andReturn(cluster).anyTimes();
+    expect(request.getProperties()).andReturn(Collections.<Map<String,Object>>emptySet());
+
+    replay(clusters, cluster, request, predicate);
+
+    provider.updateResources(request, predicate);
+
+    verify(clusters, cluster);
   }
 
   @Test
@@ -121,7 +129,10 @@ public class StageResourceProviderTest {
 
     expect(dao.findAll(request, predicate)).andReturn(entities);
 
-    replay(dao, request, predicate);
+    expect(clusters.getClusterById(anyLong())).andReturn(cluster).anyTimes();
+    expect(cluster.getClusterName()).andReturn("c1").anyTimes();
+
+    replay(dao, clusters, cluster, request, predicate);
 
     Set<Resource> resources = provider.getResources(request, predicate);
 
@@ -134,7 +145,7 @@ public class StageResourceProviderTest {
     Assert.assertEquals(1000L, resource.getPropertyValue(StageResourceProvider.STAGE_START_TIME));
     Assert.assertEquals(2500L, resource.getPropertyValue(StageResourceProvider.STAGE_END_TIME));
 
-    verify(dao);
+    verify(dao, clusters, cluster);
 
   }
 
@@ -149,7 +160,10 @@ public class StageResourceProviderTest {
 
     expect(dao.findAll(request, predicate)).andReturn(entities);
 
-    replay(dao, request, predicate);
+    expect(clusters.getClusterById(anyLong())).andReturn(cluster).anyTimes();
+    expect(cluster.getClusterName()).andReturn("c1").anyTimes();
+
+    replay(dao, clusters, cluster, request, predicate);
 
     QueryResponse response =  provider.queryForResources(request, predicate);
 
@@ -161,7 +175,7 @@ public class StageResourceProviderTest {
     Assert.assertFalse(response.isPagedResponse());
     Assert.assertEquals(1, response.getTotalResourceCount());
 
-    verify(dao);
+    verify(dao, clusters, cluster);
   }
 
   private List<StageEntity> getStageEntities() {
@@ -193,10 +207,8 @@ public class StageResourceProviderTest {
     @Override
     public void configure(Binder binder) {
       binder.bind(StageDAO.class).toInstance(dao);
-      binder.bind(Clusters.class).toInstance(
-          EasyMock.createNiceMock(Clusters.class));
-      binder.bind(Cluster.class).toInstance(
-          EasyMock.createNiceMock(Cluster.class));
+      binder.bind(Clusters.class).toInstance(clusters);
+      binder.bind(Cluster.class).toInstance(cluster);
       binder.bind(ActionMetadata.class);
     }
   }

+ 36 - 0
ambari-server/src/test/java/org/apache/ambari/server/serveraction/ServerActionExecutorTest.java

@@ -27,6 +27,7 @@ import org.apache.ambari.server.RoleCommand;
 import org.apache.ambari.server.actionmanager.*;
 import org.apache.ambari.server.agent.CommandReport;
 import org.apache.ambari.server.state.Clusters;
+import org.apache.ambari.server.serveraction.upgrades.ManualStageAction;
 import org.apache.ambari.server.state.svccomphost.ServiceComponentHostServerActionEvent;
 import org.apache.ambari.server.utils.StageUtils;
 import org.junit.BeforeClass;
@@ -83,6 +84,41 @@ public class ServerActionExecutorTest {
     assertEquals(HostRoleStatus.COMPLETED, getTaskStatus(s));
   }
 
+  /**
+   * Test a manual stage
+   */
+  @Test
+  public void testServerActionManualStage() throws Exception {
+    final Request request = createMockRequest();
+
+    final Stage stage = new Stage((long) 1, "/tmp", "cluster1", (long) 978, "context", CLUSTER_HOST_INFO,
+        "{\"host_param\":\"param_value\"}", "{\"stage_param\":\"param_value\"}");
+
+    stage.addServerActionCommand(ManualStageAction.class.getName(),
+        Role.AMBARI_SERVER_ACTION,
+        RoleCommand.EXECUTE,
+        "cluster1", SERVER_HOST_NAME,
+        new ServiceComponentHostServerActionEvent(StageUtils.getHostName(), System.currentTimeMillis()),
+        Collections.<String, String>emptyMap(), 1200);
+
+    final List<Stage> stages = new ArrayList<Stage>() {
+      {
+        add(stage);
+      }
+    };
+    ActionDBAccessor db = createMockActionDBAccessor(request, stages);
+    ServerActionExecutor.init(injector);
+    ServerActionExecutor executor = new ServerActionExecutor(db, 10000);
+
+    // Force the task to be QUEUED
+    stage.getHostRoleCommand(SERVER_HOST_NAME, Role.AMBARI_SERVER_ACTION.toString()).setStatus(HostRoleStatus.QUEUED);
+
+    int cycleCount = 0;
+    while (!getTaskStatus(stage).isHoldingState() && (cycleCount++ <= MAX_CYCLE_ITERATIONS)) {
+      executor.doWork();
+    }
+    assertEquals(HostRoleStatus.HOLDING, getTaskStatus(stage));
+  }
 
   /**
    * Test a timeout server action

+ 13 - 0
ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog200Test.java

@@ -117,6 +117,7 @@ public class UpgradeCatalog200Test {
     Capture<DBAccessor.DBColumnInfo> hostComponentStateColumnCapture = new Capture<DBAccessor.DBColumnInfo>();
     Capture<DBAccessor.DBColumnInfo> hostComponentStateSecurityStateColumnCapture = new Capture<DBAccessor.DBColumnInfo>();
     Capture<DBAccessor.DBColumnInfo> hostComponentDesiredStateSecurityStateColumnCapture = new Capture<DBAccessor.DBColumnInfo>();
+    Capture<DBAccessor.DBColumnInfo> hostRoleCommandRetryColumnCapture = new Capture<DBAccessor.DBColumnInfo>();
 
     Capture<DBAccessor.DBColumnInfo> viewparameterLabelColumnCapture = new Capture<DBAccessor.DBColumnInfo>();
     Capture<DBAccessor.DBColumnInfo> viewparameterPlaceholderColumnCapture = new Capture<DBAccessor.DBColumnInfo>();
@@ -151,6 +152,10 @@ public class UpgradeCatalog200Test {
     dbAccessor.addColumn(eq("hostcomponentstate"),
         capture(hostComponentStateColumnCapture));
 
+    // Host Role Command retry allowed
+    dbAccessor.addColumn(eq("host_role_command"),
+        capture(hostRoleCommandRetryColumnCapture));
+
     // Host Component State: security State
     dbAccessor.addColumn(eq("hostcomponentstate"),
         capture(hostComponentStateSecurityStateColumnCapture));
@@ -217,6 +222,14 @@ public class UpgradeCatalog200Test {
     assertEquals("NONE", upgradeStateColumn.getDefaultValue());
     assertFalse(upgradeStateColumn.isNullable());
 
+    // Verify added column in host_role_command table
+    DBAccessor.DBColumnInfo upgradeRetryColumn = hostRoleCommandRetryColumnCapture.getValue();
+    assertEquals("retry_allowed", upgradeRetryColumn.getName());
+    assertEquals(1, (int) upgradeRetryColumn.getLength());
+    assertEquals(Integer.class, upgradeRetryColumn.getType());
+    assertEquals(0, upgradeRetryColumn.getDefaultValue());
+    assertFalse(upgradeRetryColumn.isNullable());
+
     // verify security_state columns
     verifyComponentSecurityStateColumn(hostComponentStateSecurityStateColumnCapture);
     verifyComponentSecurityStateColumn(hostComponentDesiredStateSecurityStateColumnCapture);