Browse Source

Revert "AMBARI-8852 - RU: Cannot Retry on failure (tbeerbower)"

This reverts commit 85ff51496d555bdac09198523a9860b20be31c35.
Mahadev Konar 10 năm trước cách đây
mục cha
commit
600f1afcd9
32 tập tin đã thay đổi với 176 bổ sung356 xóa
  1. 39 43
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionScheduler.java
  2. 5 14
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java
  3. 14 59
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Stage.java
  4. 5 8
      ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariActionExecutionHelper.java
  5. 43 54
      ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
  6. 4 4
      ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
  7. 2 2
      ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
  8. 1 1
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterStackVersionResourceProvider.java
  9. 1 1
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostStackVersionResourceProvider.java
  10. 2 3
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/RequestResourceProvider.java
  11. 5 11
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StageResourceProvider.java
  12. 1 1
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeGroupResourceProvider.java
  13. 7 21
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
  14. 0 23
      ambari-server/src/main/java/org/apache/ambari/server/orm/entities/StageEntity.java
  15. 0 3
      ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog200.java
  16. 1 1
      ambari-server/src/main/java/org/apache/ambari/server/utils/StageUtils.java
  17. 0 1
      ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
  18. 0 1
      ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
  19. 0 1
      ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
  20. 0 1
      ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql
  21. 1 1
      ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql
  22. 1 1
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/ExecutionCommandWrapperTest.java
  23. 9 9
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestActionDBAccessorImpl.java
  24. 1 1
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestActionManager.java
  25. 8 8
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestActionScheduler.java
  26. 2 2
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestStage.java
  27. 4 4
      ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatHandler.java
  28. 5 5
      ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java
  29. 0 44
      ambari-server/src/test/java/org/apache/ambari/server/controller/internal/StageResourceProviderTest.java
  30. 2 2
      ambari-server/src/test/java/org/apache/ambari/server/serveraction/ServerActionExecutorTest.java
  31. 13 13
      ambari-server/src/test/java/org/apache/ambari/server/stageplanner/TestStagePlanner.java
  32. 0 13
      ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog200Test.java

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

@@ -232,11 +232,11 @@ class ActionScheduler implements Runnable {
       boolean exclusiveRequestIsGoing = false;
       // This loop greatly depends on the fact that order of stages in
       // a list does not change between invocations
-      for (Stage stage : stages) {
+      for (Stage s : stages) {
         // Check if we can process this stage in parallel with another stages
         i_stage ++;
-        long requestId = stage.getRequestId();
-        LOG.debug("==> STAGE_i = " + i_stage + "(requestId=" + requestId + ",StageId=" + stage.getStageId() + ")");
+        long requestId = s.getRequestId();
+        LOG.debug("==> STAGE_i = " + i_stage + "(requestId=" + requestId + ",StageId=" + s.getStageId() + ")");
         Request request = db.getRequest(requestId);
 
         if (request.isExclusive()) {
@@ -262,7 +262,7 @@ class ActionScheduler implements Runnable {
 
         // Commands that will be scheduled in current scheduler wakeup
         List<ExecutionCommand> commandsToSchedule = new ArrayList<ExecutionCommand>();
-        Map<String, RoleStats> roleStats = processInProgressStage(stage, commandsToSchedule);
+        Map<String, RoleStats> roleStats = processInProgressStage(s, commandsToSchedule);
         // Check if stage is failed
         boolean failed = false;
         for (Map.Entry<String, RoleStats>entry : roleStats.entrySet()) {
@@ -281,14 +281,14 @@ class ActionScheduler implements Runnable {
 
         if(!failed) {
           // Prior stage may have failed and it may need to fail the whole request
-          failed = hasPreviousStageFailed(stage);
+          failed = hasPreviousStageFailed(s);
         }
 
         if (failed) {
           LOG.warn("Operation completely failed, aborting request id:"
-              + stage.getRequestId());
-          cancelHostRoleCommands(stage.getOrderedHostRoleCommands(), FAILED_TASK_ABORT_REASONING);
-          abortOperationsForStage(stage);
+              + s.getRequestId());
+          cancelHostRoleCommands(s.getOrderedHostRoleCommands(), FAILED_TASK_ABORT_REASONING);
+          abortOperationsForStage(s);
           return;
         }
 
@@ -298,18 +298,18 @@ class ActionScheduler implements Runnable {
         //Schedule what we have so far
 
         for (ExecutionCommand cmd : commandsToSchedule) {
-            processHostRole(stage, cmd, commandsToStart, commandsToUpdate);
+            processHostRole(s, cmd, commandsToStart, commandsToUpdate);
         }
 
         LOG.debug("==> Commands to start: {}", commandsToStart.size());
         LOG.debug("==> Commands to update: {}", commandsToUpdate.size());
 
         //Multimap is analog of Map<Object, List<Object>> but allows to avoid nested loop
-        ListMultimap<String, ServiceComponentHostEvent> eventMap = formEventMap(stage, commandsToStart);
+        ListMultimap<String, ServiceComponentHostEvent> eventMap = formEventMap(s, commandsToStart);
         List<ExecutionCommand> commandsToAbort = new ArrayList<ExecutionCommand>();
         if (!eventMap.isEmpty()) {
           LOG.debug("==> processing {} serviceComponentHostEvents...", eventMap.size());
-          Cluster cluster = fsmObject.getCluster(stage.getClusterName());
+          Cluster cluster = fsmObject.getCluster(s.getClusterName());
           if (cluster != null) {
             List<ServiceComponentHostEvent> failedEvents =
               cluster.processServiceComponentHostEvents(eventMap);
@@ -327,12 +327,12 @@ class ActionScheduler implements Runnable {
               }
             }
           } else {
-            LOG.warn("There was events to process but cluster {} not found", stage.getClusterName());
+            LOG.warn("There was events to process but cluster {} not found", s.getClusterName());
           }
         }
 
         LOG.debug("==> Scheduling {} tasks...", commandsToUpdate.size());
-        db.bulkHostRoleScheduled(stage, commandsToUpdate);
+        db.bulkHostRoleScheduled(s, commandsToUpdate);
 
         if (commandsToAbort.size() > 0) { // Code branch may be a bit slow, but is extremely rarely used
           LOG.debug("==> Aborting {} tasks...", commandsToAbort.size());
@@ -344,7 +344,7 @@ class ActionScheduler implements Runnable {
           Collection<HostRoleCommand> hostRoleCommands = db.getTasks(taskIds);
 
           cancelHostRoleCommands(hostRoleCommands, FAILED_TASK_ABORT_REASONING);
-          db.bulkAbortHostRole(stage, commandsToAbort);
+          db.bulkAbortHostRole(s, commandsToAbort);
         }
 
         LOG.debug("==> Adding {} tasks to queue...", commandsToUpdate.size());
@@ -412,9 +412,7 @@ class ActionScheduler implements Runnable {
 
   private boolean hasPreviousStageFailed(Stage stage) {
     boolean failed = false;
-
     long prevStageId = stage.getStageId() - 1;
-
     if (prevStageId > 0) {
       // Find previous stage instance
       List<Stage> allStages = db.getAllStages(stage.getRequestId());
@@ -426,36 +424,34 @@ class ActionScheduler implements Runnable {
         }
       }
 
-      // If the previous stage is skippable then we shouldn't automatically fail the given stage
-      if (prevStage == null || prevStage.isSkippable()) {
-        return false;
-      }
-
-      Map<Role, Integer> hostCountsForRoles       = new HashMap<Role, Integer>();
-      Map<Role, Integer> failedHostCountsForRoles = new HashMap<Role, Integer>();
-
-      for (String host : prevStage.getHostRoleCommands().keySet()) {
-        Map<String, HostRoleCommand> roleCommandMap = prevStage.getHostRoleCommands().get(host);
-        for (String role : roleCommandMap.keySet()) {
-          HostRoleCommand c = roleCommandMap.get(role);
-          if (hostCountsForRoles.get(c.getRole()) == null) {
-            hostCountsForRoles.put(c.getRole(), 0);
-            failedHostCountsForRoles.put(c.getRole(), 0);
-          }
-          int hostCount = hostCountsForRoles.get(c.getRole());
-          hostCountsForRoles.put(c.getRole(), hostCount + 1);
-          if (c.getStatus().isFailedState()) {
-            int failedHostCount = failedHostCountsForRoles.get(c.getRole());
-            failedHostCountsForRoles.put(c.getRole(), failedHostCount + 1);
+      //It may be null for test scenarios
+      if(prevStage != null) {
+        Map<Role, Integer> hostCountsForRoles = new HashMap<Role, Integer>();
+        Map<Role, Integer> failedHostCountsForRoles = new HashMap<Role, Integer>();
+
+        for (String host : prevStage.getHostRoleCommands().keySet()) {
+          Map<String, HostRoleCommand> roleCommandMap = prevStage.getHostRoleCommands().get(host);
+          for (String role : roleCommandMap.keySet()) {
+            HostRoleCommand c = roleCommandMap.get(role);
+            if (hostCountsForRoles.get(c.getRole()) == null) {
+              hostCountsForRoles.put(c.getRole(), 0);
+              failedHostCountsForRoles.put(c.getRole(), 0);
+            }
+            int hostCount = hostCountsForRoles.get(c.getRole());
+            hostCountsForRoles.put(c.getRole(), hostCount + 1);
+            if (c.getStatus().isFailedState()) {
+              int failedHostCount = failedHostCountsForRoles.get(c.getRole());
+              failedHostCountsForRoles.put(c.getRole(), failedHostCount + 1);
+            }
           }
         }
-      }
 
-      for (Role role : hostCountsForRoles.keySet()) {
-        float failedHosts = failedHostCountsForRoles.get(role);
-        float totalHosts = hostCountsForRoles.get(role);
-        if (((totalHosts - failedHosts) / totalHosts) < prevStage.getSuccessFactor(role)) {
-          failed = true;
+        for (Role role : hostCountsForRoles.keySet()) {
+          float failedHosts = failedHostCountsForRoles.get(role);
+          float totalHosts = hostCountsForRoles.get(role);
+          if (((totalHosts - failedHosts) / totalHosts) < prevStage.getSuccessFactor(role)) {
+            failed = true;
+          }
         }
       }
     }

+ 5 - 14
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java

@@ -26,6 +26,8 @@ import org.apache.ambari.server.orm.dao.ExecutionCommandDAO;
 import org.apache.ambari.server.orm.entities.ExecutionCommandEntity;
 import org.apache.ambari.server.orm.entities.HostRoleCommandEntity;
 import org.apache.ambari.server.state.ServiceComponentHostEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class encapsulates the information for an task on a host for a
@@ -34,6 +36,7 @@ import org.apache.ambari.server.state.ServiceComponentHostEvent;
  * track the request.
  */
 public class HostRoleCommand {
+  private static final Logger log = LoggerFactory.getLogger(HostRoleCommand.class);
   private final Role role;
   private final ServiceComponentHostEventWrapper event;
   private long taskId = -1;
@@ -51,7 +54,6 @@ public class HostRoleCommand {
   private long endTime = -1;
   private long lastAttemptTime = -1;
   private short attemptCount = 0;
-  private final boolean retryAllowed;
   private RoleCommand roleCommand;
   private String commandDetail;
   private String customCommandName;
@@ -60,16 +62,10 @@ public class HostRoleCommand {
 
   public HostRoleCommand(String host, Role role,
                          ServiceComponentHostEvent event, RoleCommand command) {
-    this(host, role, event, command, false);
-  }
-
-  public HostRoleCommand(String host, Role role,
-                         ServiceComponentHostEvent event, RoleCommand command, boolean retryAllowed) {
     this.hostName = host;
     this.role = role;
     this.event = new ServiceComponentHostEventWrapper(event);
     this.roleCommand = command;
-    this.retryAllowed = retryAllowed;
   }
 
   @AssistedInject
@@ -90,7 +86,6 @@ public class HostRoleCommand {
     endTime = hostRoleCommandEntity.getEndTime() != null ? hostRoleCommandEntity.getEndTime() : -1L;
     lastAttemptTime = hostRoleCommandEntity.getLastAttemptTime();
     attemptCount = hostRoleCommandEntity.getAttemptCount();
-    retryAllowed = hostRoleCommandEntity.isRetryAllowed();
     roleCommand = hostRoleCommandEntity.getRoleCommand();
     event = new ServiceComponentHostEventWrapper(hostRoleCommandEntity.getEvent());
     commandDetail = hostRoleCommandEntity.getCommandDetail();
@@ -113,7 +108,6 @@ public class HostRoleCommand {
     hostRoleCommandEntity.setEndTime(endTime);
     hostRoleCommandEntity.setLastAttemptTime(lastAttemptTime);
     hostRoleCommandEntity.setAttemptCount(attemptCount);
-    hostRoleCommandEntity.setRetryAllowed(retryAllowed);
     hostRoleCommandEntity.setRoleCommand(roleCommand);
     hostRoleCommandEntity.setCommandDetail(commandDetail);
     hostRoleCommandEntity.setCustomCommandName(customCommandName);
@@ -239,10 +233,6 @@ public class HostRoleCommand {
     this.attemptCount++;
   }
 
-  public boolean isRetryAllowed() {
-    return retryAllowed;
-  }
-
   public String getStructuredOut() {
     return structuredOut;
   }
@@ -295,7 +285,8 @@ public class HostRoleCommand {
 
   @Override
   public int hashCode() {
-    return (hostName + role.toString() + roleCommand.toString()).hashCode();
+    return (hostName.toString() + role.toString() + roleCommand.toString())
+        .hashCode();
   }
 
   @Override

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

@@ -65,8 +65,6 @@ public class Stage {
   private String commandParamsStage;
   private String hostParamsStage;
 
-  private boolean skippable;
-
   private int stageTimeout = -1;
 
   private volatile boolean wrappersLoaded = false;
@@ -98,7 +96,6 @@ public class Stage {
     this.clusterHostInfo = clusterHostInfo;
     this.commandParamsStage = commandParamsStage;
     this.hostParamsStage = hostParamsStage;
-    this.skippable = false;
   }
 
   @AssistedInject
@@ -107,7 +104,6 @@ public class Stage {
 
     requestId = stageEntity.getRequestId();
     stageId = stageEntity.getStageId();
-    skippable = stageEntity.isSkippable();
     logDir = stageEntity.getLogInfo();
 
     long clusterId = stageEntity.getClusterId().longValue();
@@ -154,7 +150,6 @@ public class Stage {
     stageEntity.setRequestId(requestId);
     stageEntity.setStageId(getStageId());
     stageEntity.setLogInfo(logDir);
-    stageEntity.setSkippable(skippable);
     stageEntity.setRequestContext(requestContext);
     stageEntity.setHostRoleCommands(new ArrayList<HostRoleCommandEntity>());
     stageEntity.setRoleSuccessCriterias(new ArrayList<RoleSuccessCriteriaEntity>());
@@ -252,12 +247,9 @@ public class Stage {
     return StageUtils.getActionId(requestId, getStageId());
   }
 
-  private synchronized ExecutionCommandWrapper addGenericExecutionCommand(
-      String clusterName, String hostName, Role role,
-      RoleCommand command, ServiceComponentHostEvent event, boolean retryAllowed){
-
+  private synchronized ExecutionCommandWrapper addGenericExecutionCommand(String clusterName, String hostName, Role role, RoleCommand command, ServiceComponentHostEvent event){
     //used on stage creation only, no need to check if wrappers loaded
-    HostRoleCommand hrc = new HostRoleCommand(hostName, role, event, command, retryAllowed);
+    HostRoleCommand hrc = new HostRoleCommand(hostName, role, event, command);
     ExecutionCommand cmd = new ExecutionCommand();
     ExecutionCommandWrapper wrapper = new ExecutionCommandWrapper(cmd);
     hrc.setExecutionCommandWrapper(wrapper);
@@ -295,23 +287,20 @@ public class Stage {
     execCmdList.add(wrapper);
     return wrapper;
   }
-
   /**
    * A new host role command is created for execution.
    * Creates both ExecutionCommand and HostRoleCommand objects and
    * adds them to the Stage. This should be called only once for a host-role
    * for a given stage.
    */
-  public synchronized void addHostRoleExecutionCommand(String host, Role role, RoleCommand command,
-                                                       ServiceComponentHostEvent event, String clusterName,
-                                                       String serviceName, boolean retryAllowed) {
-
-    ExecutionCommandWrapper commandWrapper =
-        addGenericExecutionCommand(clusterName, host, role, command, event, retryAllowed);
+  public synchronized void addHostRoleExecutionCommand(String host, Role role,  RoleCommand command,
+      ServiceComponentHostEvent event, String clusterName, String serviceName) {
+    ExecutionCommandWrapper commandWrapper = addGenericExecutionCommand(clusterName, host, role, command, event);
 
     commandWrapper.getExecutionCommand().setServiceName(serviceName);
   }
 
+
   /**
    * Creates server-side execution command.
    * <p/>
@@ -329,17 +318,15 @@ public class Stage {
    * @param commandDetail a String declaring a descriptive name to pass to the action - null or an
    *                      empty string indicates no value is to be set
    * @param timeout       an Integer declaring the timeout for this action - if null, a default
-   * @param retryAllowed   indicates whether retry after failure is allowed
    */
   public synchronized void addServerActionCommand(String actionName, Role role, RoleCommand command,
                                                   String clusterName, ServiceComponentHostServerActionEvent event,
                                                   @Nullable Map<String, String> commandParams,
                                                   @Nullable String commandDetail,
-                                                  @Nullable Integer timeout,
-                                                  boolean retryAllowed) {
+                                                  @Nullable Integer timeout) {
 
     addServerActionCommand(actionName, role, command,
-        clusterName, StageUtils.getHostName(), event, commandParams, commandDetail, timeout, retryAllowed);
+        clusterName, StageUtils.getHostName(), event, commandParams, commandDetail, timeout);
   }
 
   /**
@@ -364,17 +351,14 @@ public class Stage {
    * @param commandDetail a String declaring a descriptive name to pass to the action - null or an
    *                      empty string indicates no value is to be set
    * @param timeout       an Integer declaring the timeout for this action - if null, a default
-   * @param retryAllowed   indicates whether retry after failure is allowed
    */
   public synchronized void addServerActionCommand(String actionName, Role role, RoleCommand command,
                                                   String clusterName, String hostName,
                                                   ServiceComponentHostServerActionEvent event,
                                                   @Nullable Map<String, String> commandParams,
                                                   @Nullable String commandDetail,
-                                                  @Nullable Integer timeout, boolean retryAllowed) {
-    ExecutionCommandWrapper commandWrapper =
-        addGenericExecutionCommand(clusterName, hostName, role, command, event, retryAllowed);
-
+                                                  @Nullable Integer timeout) {
+    ExecutionCommandWrapper commandWrapper = addGenericExecutionCommand(clusterName, hostName, role, command, event);
     ExecutionCommand cmd = commandWrapper.getExecutionCommand();
 
     Map<String, String> cmdParams = new HashMap<String, String>();
@@ -400,12 +384,10 @@ public class Stage {
   }
 
   /**
-   *  Adds cancel command to stage for given cancelTargets collection of
-   *  task id's that has to be canceled in Agent layer.
+   *  Adds cancel command to stage for given cancelTargets collection of task id's that has to be canceled in Agent layer.
    */
   public synchronized void addCancelRequestCommand(List<Long> cancelTargets, String clusterName, String hostName) {
-    ExecutionCommandWrapper commandWrapper = addGenericExecutionCommand(clusterName, hostName,
-        Role.AMBARI_SERVER_ACTION, RoleCommand.ABORT, null, false);
+    ExecutionCommandWrapper commandWrapper = addGenericExecutionCommand(clusterName, hostName, Role.AMBARI_SERVER_ACTION, RoleCommand.ABORT, null);
     ExecutionCommand cmd = commandWrapper.getExecutionCommand();
     cmd.setCommandType(AgentCommandType.CANCEL_COMMAND);
 
@@ -587,7 +569,8 @@ public class Stage {
   /**
    * This method should be used only in stage planner. To add
    * a new execution command use
-   * {@link #addHostRoleExecutionCommand(String, org.apache.ambari.server.Role, org.apache.ambari.server.RoleCommand, org.apache.ambari.server.state.ServiceComponentHostEvent, String, String, boolean)}
+   * {@link #addHostRoleExecutionCommand(String, Role, RoleCommand,
+   * ServiceComponentHostEvent, String, String)}
    */
   public synchronized void addExecutionCommandWrapper(Stage origStage,
       String hostname, Role r) {
@@ -640,34 +623,6 @@ public class Stage {
     return stageTimeout;
   }
 
-  /**
-   * Determine whether or not this stage is skippable.
-   *
-   * A skippable stage can be skipped on failure so that the
-   * remaining stages of the request can execute.
-   * If a stage is not skippable, a failure will cause the
-   * remaining stages of the request to be aborted.
-   *
-   * @return true if this stage is skippable
-   */
-  public boolean isSkippable() {
-    return skippable;
-  }
-
-  /**
-   * Set skippable for this stage.
-   *
-   * A skippable stage can be skipped on failure so that the
-   * remaining stages of the request can execute.
-   * If a stage is not skippable, a failure will cause the
-   * remaining stages of the request to be aborted.
-   *
-   * @param skippable  true if this stage should be skippable
-   */
-  public void setSkippable(boolean skippable) {
-    this.skippable = skippable;
-  }
-
   @Override //Object
   public synchronized String toString() {
     StringBuilder builder = new StringBuilder();

+ 5 - 8
ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariActionExecutionHelper.java

@@ -202,15 +202,12 @@ public class AmbariActionExecutionHelper {
 
   /**
    * Add tasks to the stage based on the requested action execution
-   *
-   * @param actionContext  the context associated with the action
-   * @param stage          stage into which tasks must be inserted
-   * @param retryAllowed   indicates whether retry is allowed on failure
-   *
-   * @throws AmbariException if the task can not be added
+   * @param actionContext the context associated with the action
+   * @param stage stage into which tasks must be inserted
+   * @throws AmbariException
    */
   public void addExecutionCommandsToStage(
-      final ActionExecutionContext actionContext, Stage stage, boolean retryAllowed)
+          final ActionExecutionContext actionContext, Stage stage)
       throws AmbariException {
 
     String actionName = actionContext.getActionName();
@@ -334,7 +331,7 @@ public class AmbariActionExecutionHelper {
         Role.valueOf(actionContext.getActionName()), RoleCommand.ACTIONEXECUTE,
           new ServiceComponentHostOpInProgressEvent(actionContext.getActionName(),
             hostName, System.currentTimeMillis()), clusterName,
-              serviceName, retryAllowed);
+              serviceName);
 
       Map<String, Map<String, String>> configurations = new TreeMap<String, Map<String, String>>();
       Map<String, Map<String, Map<String, String>>> configurationAttributes = new TreeMap<String, Map<String, Map<String, String>>>();

+ 43 - 54
ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java

@@ -112,14 +112,12 @@ public class AmbariCustomCommandExecutionHelper {
     masterToSlaveMappingForDecom.put("JOBTRACKER", "TASKTRACKER");
   }
 
-  public final static String DECOM_INCLUDED_HOSTS = "included_hosts";
-  public final static String DECOM_EXCLUDED_HOSTS = "excluded_hosts";
-  public final static String DECOM_SLAVE_COMPONENT = "slave_type";
-  public final static String HBASE_MARK_DRAINING_ONLY = "mark_draining_only";
-  public final static String UPDATE_EXCLUDE_FILE_ONLY = "update_exclude_file_only";
-
-  private final static String ALIGN_MAINTENANCE_STATE = "align_maintenance_state";
-
+  public static String DECOM_INCLUDED_HOSTS = "included_hosts";
+  public static String DECOM_EXCLUDED_HOSTS = "excluded_hosts";
+  public static String DECOM_SLAVE_COMPONENT = "slave_type";
+  public static String HBASE_MARK_DRAINING_ONLY = "mark_draining_only";
+  public static String UPDATE_EXCLUDE_FILE_ONLY = "update_exclude_file_only";
+  private static String ALIGN_MAINTENANCE_STATE = "align_maintenance_state";
   @Inject
   private ActionMetadata actionMetadata;
   @Inject
@@ -145,8 +143,15 @@ public class AmbariCustomCommandExecutionHelper {
 
   private Boolean isServiceCheckCommand(String command, String service) {
     List<String> actions = actionMetadata.getActions(service);
+    if (actions == null || actions.size() == 0) {
+      return false;
+    }
 
-    return !(actions == null || actions.size() == 0) && actions.contains(command);
+    if (!actions.contains(command)) {
+      return false;
+    }
+
+    return true;
   }
 
   private Boolean isValidCustomCommand(String clusterName,
@@ -219,8 +224,7 @@ public class AmbariCustomCommandExecutionHelper {
                                       final RequestResourceFilter resourceFilter,
                                       Stage stage,
                                       Map<String, String> additionalCommandParams,
-                                      String commandDetail,
-                                      boolean retryAllowed)
+                                      String commandDetail)
                                       throws AmbariException {
     final String serviceName = resourceFilter.getServiceName();
     final String componentName = resourceFilter.getComponentName();
@@ -283,7 +287,7 @@ public class AmbariCustomCommandExecutionHelper {
       stage.addHostRoleExecutionCommand(hostName, Role.valueOf(componentName),
           RoleCommand.CUSTOM_COMMAND,
           new ServiceComponentHostOpInProgressEvent(componentName,
-              hostName, nowTimestamp), cluster.getClusterName(), serviceName, retryAllowed);
+              hostName, nowTimestamp), cluster.getClusterName(), serviceName);
 
       Map<String, Map<String, String>> configurations =
           new TreeMap<String, Map<String, String>>();
@@ -383,11 +387,10 @@ public class AmbariCustomCommandExecutionHelper {
   }
 
   /**
-   * Splits the passed comma separated value and returns it as set.
-   *
-   * @param commaSeparatedTags  separated list
-   *
+   * splits the passed commaseparated value and returns it as set
+   * @param commaSeparatedTags separated list
    * @return set of items or null
+   * @throws AmbariException
    */
   private Set<String> parseAndValidateComponentsMapping(String commaSeparatedTags) {
     Set<String> retVal = null;
@@ -398,10 +401,9 @@ public class AmbariCustomCommandExecutionHelper {
   }
 
   private void findHostAndAddServiceCheckAction(
-      final ActionExecutionContext actionExecutionContext,
-      final RequestResourceFilter resourceFilter,
-      Stage stage,
-      boolean retryAllowed)
+          final ActionExecutionContext actionExecutionContext,
+          final RequestResourceFilter resourceFilter,
+          Stage stage)
           throws AmbariException {
 
     String clusterName = actionExecutionContext.getClusterName();
@@ -470,7 +472,7 @@ public class AmbariCustomCommandExecutionHelper {
     }
 
     addServiceCheckAction(stage, hostName, smokeTestRole, nowTimestamp,
-        serviceName, componentName, actionParameters, retryAllowed);
+        serviceName, componentName, actionParameters);
   }
 
   /**
@@ -483,8 +485,7 @@ public class AmbariCustomCommandExecutionHelper {
                                     long nowTimestamp,
                                     String serviceName,
                                     String componentName,
-                                    Map<String, String> actionParameters,
-                                    boolean retryAllowed)
+                                    Map<String, String> actionParameters)
                                     throws AmbariException {
 
     String clusterName = stage.getClusterName();
@@ -502,7 +503,7 @@ public class AmbariCustomCommandExecutionHelper {
         Role.valueOf(smokeTestRole),
         RoleCommand.SERVICE_CHECK,
         new ServiceComponentHostOpInProgressEvent(componentName, hostname,
-            nowTimestamp), cluster.getClusterName(), serviceName, retryAllowed);
+            nowTimestamp), cluster.getClusterName(), serviceName);
 
     HostRoleCommand hrc = stage.getHostRoleCommand(hostname, smokeTestRole);
     if (hrc != null) {
@@ -582,8 +583,7 @@ public class AmbariCustomCommandExecutionHelper {
    */
   private void addDecommissionAction(final ActionExecutionContext actionExecutionContext,
                                      final RequestResourceFilter resourceFilter,
-                                     Stage stage,
-                                     boolean retryAllowed)
+                                     Stage stage)
                                      throws AmbariException {
 
     String clusterName = actionExecutionContext.getClusterName();
@@ -794,7 +794,7 @@ public class AmbariCustomCommandExecutionHelper {
       if (!serviceName.equals(Service.Type.HBASE.name()) || hostName.equals(primaryCandidate)) {
         commandParams.put(UPDATE_EXCLUDE_FILE_ONLY, "false");
         addCustomCommandAction(commandContext, commandFilter, stage,
-          commandParams, commandDetail.toString(), retryAllowed);
+          commandParams, commandDetail.toString());
       }
     }
   }
@@ -816,10 +816,8 @@ public class AmbariCustomCommandExecutionHelper {
 
   /**
    * Validate custom command and throw exception is invalid request.
-   *
-   * @param actionRequest  the action request
-   *
-   * @throws AmbariException if the action can not be validated
+   * @param actionRequest
+   * @throws AmbariException
    */
   public void validateAction(ExecuteActionRequest actionRequest) throws AmbariException {
 
@@ -853,17 +851,13 @@ public class AmbariCustomCommandExecutionHelper {
 
   /**
    * Other than Service_Check and Decommission all other commands are pass-through
-   *
-   * @param actionExecutionContext  received request to execute a command
-   * @param stage                   the initial stage for task creation
-   * @param retryAllowed            indicates whether the the command allows retry
-   *
-   * @throws AmbariException if the commands can not be added
+   * @param actionExecutionContext received request to execute a command
+   * @param stage the initial stage for task creation
+   * @throws AmbariException
    */
   public void addExecutionCommandsToStage(ActionExecutionContext actionExecutionContext,
                                           Stage stage,
-                                          Map<String, String> requestParams,
-                                          boolean retryAllowed)
+                                          Map<String, String> requestParams)
                                           throws AmbariException {
 
     List<RequestResourceFilter> resourceFilters = actionExecutionContext.getResourceFilters();
@@ -876,9 +870,9 @@ public class AmbariCustomCommandExecutionHelper {
 
       if (actionExecutionContext.getActionName().contains(SERVICE_CHECK_COMMAND_NAME)) {
         findHostAndAddServiceCheckAction(actionExecutionContext,
-          resourceFilter, stage, retryAllowed);
+          resourceFilter, stage);
       } else if (actionExecutionContext.getActionName().equals(DECOMMISSION_COMMAND_NAME)) {
-        addDecommissionAction(actionExecutionContext, resourceFilter, stage, retryAllowed);
+        addDecommissionAction(actionExecutionContext, resourceFilter, stage);
       } else if (isValidCustomCommand(actionExecutionContext, resourceFilter)) {
         String commandDetail = getReadableCustomCommandDetail(actionExecutionContext, resourceFilter);
 
@@ -895,7 +889,7 @@ public class AmbariCustomCommandExecutionHelper {
           actionExecutionContext.getParameters().put(KeyNames.REFRESH_ADITIONAL_COMPONENT_TAGS, requestParams.get(KeyNames.REFRESH_ADITIONAL_COMPONENT_TAGS));
         }
         addCustomCommandAction(actionExecutionContext, resourceFilter, stage,
-          extraParams, commandDetail, retryAllowed);
+          extraParams, commandDetail);
       } else {
         throw new AmbariException("Unsupported action " +
           actionExecutionContext.getActionName());
@@ -905,13 +899,10 @@ public class AmbariCustomCommandExecutionHelper {
 
   /**
    * Get repository info given a cluster and host.
-   *
-   * @param cluster  the cluster
-   * @param host     the host
-   *
+   * @param cluster
+   * @param host
    * @return the repo info
-   *
-   * @throws AmbariException if the repository information can not be obtained
+   * @throws AmbariException
    */
   public String getRepoInfo(Cluster cluster, Host host) throws AmbariException {
     StackId stackId = cluster.getDesiredStackVersion();
@@ -942,12 +933,10 @@ public class AmbariCustomCommandExecutionHelper {
 
 
   /**
-   * Helper method to fill execution command information.
-   *
-   * @param actionExecContext  the context
-   * @param cluster            the cluster for the command
-   *
-   * @return a wrapper of the imporant JSON structures to add to a stage
+   * Helper method to fill execution command information
+   * @param actionExecutionContext the context
+   * @param cluster the cluster for the command
+   * @return a wrapper of the imporant JSON structures to add to a stage.
    */
   public ExecuteCommandJson getCommandJson(ActionExecutionContext actionExecContext,
       Cluster cluster) throws AmbariException {

+ 4 - 4
ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java

@@ -1593,7 +1593,7 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
     stage.addHostRoleExecutionCommand(scHost.getHostName(), Role.valueOf(scHost
       .getServiceComponentName()), roleCommand,
       event, scHost.getClusterName(),
-      scHost.getServiceName(), false);
+      scHost.getServiceName());
     String serviceName = scHost.getServiceName();
     String componentName = event.getServiceComponentName();
     String hostname = scHost.getHostName();
@@ -2073,7 +2073,7 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
 
         customCommandExecutionHelper.addServiceCheckAction(stage, clientHost,
           smokeTestRole, nowTimestamp, serviceName,
-          componentName, null, false);
+          componentName, null);
       }
 
       RoleCommandOrder rco = getRoleCommandOrder(cluster);
@@ -2846,9 +2846,9 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
         jsons.getHostParamsForStage());
 
     if (actionRequest.isCommand()) {
-      customCommandExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage, requestProperties, false);
+      customCommandExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage, requestProperties);
     } else {
-      actionExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage, false);
+      actionExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage);
     }
 
     RoleGraph rg;

+ 2 - 2
ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java

@@ -763,7 +763,7 @@ public class KerberosHelper {
         event,
         commandParameters,
         commandDetail,
-        timeout, false);
+        timeout);
 
     return stage;
   }
@@ -1049,7 +1049,7 @@ public class KerberosHelper {
             "SET_KEYTAB",
             requestResourceFilters,
             requestParams);
-        customCommandExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage, requestParams, false);
+        customCommandExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage, requestParams);
       }
 
       roleGraph = new RoleGraph(roleCommandOrder);

+ 1 - 1
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterStackVersionResourceProvider.java

@@ -363,7 +363,7 @@ public class ClusterStackVersionResourceProvider extends AbstractControllerResou
       actionContext.setTimeout(Short.valueOf(configuration.getDefaultAgentTaskTimeout()));
 
       try {
-        actionExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage, false);
+        actionExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage);
       } catch (AmbariException e) {
         throw new SystemException("Can not modify stage", e);
       }

+ 1 - 1
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostStackVersionResourceProvider.java

@@ -412,7 +412,7 @@ public class HostStackVersionResourceProvider extends AbstractControllerResource
     req.addStages(Collections.singletonList(stage));
 
     try {
-      actionExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage, false);
+      actionExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage);
     } catch (AmbariException e) {
       throw new SystemException("Can not modify stage", e);
     }

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

@@ -222,7 +222,7 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
       List<HostRoleCommand> commands = internalRequest.getCommands();
       HostRoleStatus internalRequestStatus =
           StageResourceProvider.calculateSummaryStatus(
-              StageResourceProvider.calculateTaskStatusCounts(getHostRoleStatuses(commands)), commands.size(), true);
+              StageResourceProvider.calculateTaskStatusCounts(getHostRoleStatuses(commands)), commands.size());
 
       if (updateRequest.getStatus() != HostRoleStatus.ABORTED) {
         throw new IllegalArgumentException(
@@ -471,8 +471,7 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
     Map<HostRoleStatus, Integer> hostRoleStatusCounters =
         StageResourceProvider.calculateTaskStatusCounts(getHostRoleStatuses(commands));
 
-    HostRoleStatus requestStatus =
-        StageResourceProvider.calculateSummaryStatus(hostRoleStatusCounters, taskCount, true);
+    HostRoleStatus requestStatus   = StageResourceProvider.calculateSummaryStatus(hostRoleStatusCounters, taskCount);
 
     double progressPercent = StageResourceProvider.calculateProgressPercent(hostRoleStatusCounters, taskCount);
 

+ 5 - 11
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StageResourceProvider.java

@@ -89,7 +89,6 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
   public static final String STAGE_CLUSTER_HOST_INFO = "Stage/cluster_host_info";
   public static final String STAGE_COMMAND_PARAMS = "Stage/command_params";
   public static final String STAGE_HOST_PARAMS = "Stage/host_params";
-  public static final String STAGE_SKIPPABLE = "Stage/skippable";
   public static final String STAGE_PROGRESS_PERCENT = "Stage/progress_percent";
   public static final String STAGE_STATUS = "Stage/status";
   public static final String STAGE_START_TIME = "Stage/start_time";
@@ -116,7 +115,6 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
     PROPERTY_IDS.add(STAGE_CLUSTER_HOST_INFO);
     PROPERTY_IDS.add(STAGE_COMMAND_PARAMS);
     PROPERTY_IDS.add(STAGE_HOST_PARAMS);
-    PROPERTY_IDS.add(STAGE_SKIPPABLE);
     PROPERTY_IDS.add(STAGE_PROGRESS_PERCENT);
     PROPERTY_IDS.add(STAGE_STATUS);
     PROPERTY_IDS.add(STAGE_START_TIME);
@@ -267,7 +265,7 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
 
     Map<HostRoleStatus, Integer> taskStatusCounts = calculateTaskStatusCounts(getHostRoleStatuses(tasks));
 
-    HostRoleStatus currentStatus = calculateSummaryStatus(taskStatusCounts, tasks.size(), !entity.isSkippable());
+    HostRoleStatus currentStatus = calculateSummaryStatus(taskStatusCounts, tasks.size());
 
     if (!isValidManualTransition(currentStatus, desiredStatus)) {
       throw new IllegalArgumentException("Can not transition a stage from " +
@@ -318,7 +316,6 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
     setResourceProperty(resource, STAGE_CLUSTER_HOST_INFO, entity.getClusterHostInfo(), requestedIds);
     setResourceProperty(resource, STAGE_COMMAND_PARAMS, entity.getCommandParamsStage(), requestedIds);
     setResourceProperty(resource, STAGE_HOST_PARAMS, entity.getHostParamsStage(), requestedIds);
-    setResourceProperty(resource, STAGE_SKIPPABLE, entity.isSkippable(), requestedIds);
 
     Collection<HostRoleCommandEntity> tasks = entity.getHostRoleCommands();
 
@@ -339,8 +336,7 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
 
     setResourceProperty(resource, STAGE_PROGRESS_PERCENT, calculateProgressPercent(taskStatusCounts, taskCount),
         requestedIds);
-    setResourceProperty(resource, STAGE_STATUS,
-        calculateSummaryStatus(taskStatusCounts, taskCount, !entity.isSkippable()).toString(),
+    setResourceProperty(resource, STAGE_STATUS, calculateSummaryStatus(taskStatusCounts, taskCount).toString(),
         requestedIds);
 
     return resource;
@@ -368,18 +364,16 @@ public class StageResourceProvider extends AbstractResourceProvider implements E
    *
    * @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) {
+  protected static HostRoleStatus calculateSummaryStatus(Map<HostRoleStatus, Integer> counters, int total) {
     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 && failAll ? HostRoleStatus.FAILED :
+        counters.get(HostRoleStatus.FAILED) > 0 ? HostRoleStatus.FAILED :
         counters.get(HostRoleStatus.ABORTED) > 0 ? HostRoleStatus.ABORTED :
-        counters.get(HostRoleStatus.TIMEDOUT) > 0 && failAll ? HostRoleStatus.TIMEDOUT :
+        counters.get(HostRoleStatus.TIMEDOUT) > 0 ? HostRoleStatus.TIMEDOUT :
         counters.get(HostRoleStatus.IN_PROGRESS) > 0 ? HostRoleStatus.IN_PROGRESS :
         counters.get(HostRoleStatus.COMPLETED) == total ? HostRoleStatus.COMPLETED : HostRoleStatus.PENDING;
   }

+ 1 - 1
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeGroupResourceProvider.java

@@ -190,7 +190,7 @@ public class UpgradeGroupResourceProvider extends AbstractControllerResourceProv
         StageResourceProvider.calculateTaskStatusCounts(getHostRoleStatuses(stages));
 
     setResourceProperty(upgradeGroup, UPGRADE_GROUP_STATUS,
-        StageResourceProvider.calculateSummaryStatus(counters, stages.size(), false), requestedIds);
+        StageResourceProvider.calculateSummaryStatus(counters, stages.size()), requestedIds);
 
     setResourceProperty(upgradeGroup, UPGRADE_GROUP_PROGRESS_PERCENT,
         StageResourceProvider.calculateProgressPercent(counters, stages.size()), requestedIds);

+ 7 - 21
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java

@@ -88,12 +88,6 @@ import com.google.inject.Provider;
 @StaticallyInject
 public class UpgradeResourceProvider extends AbstractControllerResourceProvider {
 
-  /**
-   * Default failure retry/skip options for upgrades.
-   */
-  private static final boolean UPGRADE_DEFAULT_ALLOW_RETRY = true;
-  private static final boolean UPGRADE_DEFAULT_SKIPPABLE = true;
-
   protected static final String UPGRADE_CLUSTER_NAME = "Upgrade/cluster_name";
   protected static final String UPGRADE_VERSION = "Upgrade/repository_version";
   protected static final String UPGRADE_REQUEST_ID = "Upgrade/request_id";
@@ -144,7 +138,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
   /**
    * Constructor.
    *
-   * @param controller  the controller
+   * @param controller
    */
   UpgradeResourceProvider(AmbariManagementController controller) {
     super(PROPERTY_IDS, KEY_PROPERTY_IDS, controller);
@@ -171,7 +165,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
           UpgradePack up = validateRequest(requestMap);
 
           return createUpgrade(up, requestMap);
-        }
+        };
       });
 
     notifyCreate(Resource.Type.Upgrade, request);
@@ -196,7 +190,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
         throw new IllegalArgumentException("The cluster name is required when querying for upgrades");
       }
 
-      Cluster cluster;
+      Cluster cluster = null;
       try {
         cluster = getManagementController().getClusters().getCluster(clusterName);
       } catch (AmbariException e) {
@@ -505,8 +499,6 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
         "{}", "{}",
         StageUtils.getGson().toJson(hostLevelParams));
 
-    stage.setSkippable(UPGRADE_DEFAULT_SKIPPABLE);
-
     long stageId = request.getLastStageId() + 1;
     if (0L == stageId) {
       stageId = 1L;
@@ -516,7 +508,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
 
     // !!! TODO verify the action is valid
 
-    actionExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage, UPGRADE_DEFAULT_ALLOW_RETRY);
+    actionExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage);
 
     // need to set meaningful text on the command
     for (Map<String, HostRoleCommand> map : stage.getHostRoleCommands().values()) {
@@ -561,8 +553,6 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
         jsons.getCommandParamsForStage(),
         jsons.getHostParamsForStage());
 
-    stage.setSkippable(UPGRADE_DEFAULT_SKIPPABLE);
-
     long stageId = request.getLastStageId() + 1;
     if (0L == stageId) {
       stageId = 1L;
@@ -575,7 +565,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
     Map<String, String> requestParams = new HashMap<String, String>();
     requestParams.put("command", "RESTART");
 
-    commandExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage, requestParams, UPGRADE_DEFAULT_ALLOW_RETRY);
+    commandExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage, requestParams);
 
     request.addStages(Collections.singletonList(stage));
   }
@@ -610,8 +600,6 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
         jsons.getCommandParamsForStage(),
         jsons.getHostParamsForStage());
 
-    stage.setSkippable(UPGRADE_DEFAULT_SKIPPABLE);
-
     long stageId = request.getLastStageId() + 1;
     if (0L == stageId) {
       stageId = 1L;
@@ -621,7 +609,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
 
     Map<String, String> requestParams = new HashMap<String, String>();
 
-    commandExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage, requestParams, UPGRADE_DEFAULT_ALLOW_RETRY);
+    commandExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage, requestParams);
 
     request.addStages(Collections.singletonList(stage));
   }
@@ -671,8 +659,6 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
         jsons.getCommandParamsForStage(),
         jsons.getHostParamsForStage());
 
-    stage.setSkippable(UPGRADE_DEFAULT_SKIPPABLE);
-
     long stageId = request.getLastStageId() + 1;
     if (0L == stageId) {
       stageId = 1L;
@@ -688,7 +674,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
         RoleCommand.EXECUTE,
         cluster.getClusterName(), host,
         new ServiceComponentHostServerActionEvent(StageUtils.getHostName(), System.currentTimeMillis()),
-        commandParams, null, 1200, UPGRADE_DEFAULT_ALLOW_RETRY);
+        commandParams, null, 1200);
 
     request.addStages(Collections.singletonList(stage));
   }

+ 0 - 23
ambari-server/src/main/java/org/apache/ambari/server/orm/entities/StageEntity.java

@@ -40,9 +40,6 @@ public class StageEntity {
   @Id
   private Long stageId = 0L;
 
-  @Column(name = "skippable", nullable = false)
-  private Integer skippable = Integer.valueOf(0);
-
   @Column(name = "log_info")
   @Basic
   private String logInfo = "";
@@ -194,24 +191,4 @@ public class StageEntity {
   public void setRequest(RequestEntity request) {
     this.request = request;
   }
-
-  /**
-   * Determine whether this stage is skippable.  If the stage is skippable then in can be skipped on
-   * error without failing the entire request.
-   *
-   * @return true if this stage is skippable
-   */
-  public boolean isSkippable() {
-    return skippable != 0;
-  }
-
-  /**
-   * Set skippable for this stage.  If the stage is skippable then in can be skipped on
-   * error without failing the entire request.
-   *
-   * @param skippable true indicates that the stage is skippable
-   */
-  public void setSkippable(boolean skippable) {
-    this.skippable = skippable ? 1 : 0;
-  }
 }

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

@@ -176,9 +176,6 @@ public class UpgradeCatalog200 extends AbstractUpgradeCatalog {
     dbAccessor.addColumn("host_role_command", new DBAccessor.DBColumnInfo("retry_allowed",
         Integer.class, 1, 0, false));
 
-    dbAccessor.addColumn("stage", new DBAccessor.DBColumnInfo("skippable",
-        Integer.class, 1, 0, false));
-
     // New tables
     columns = new ArrayList<DBColumnInfo>();
     columns.add(new DBAccessor.DBColumnInfo("id", Long.class, null, null, false));

+ 1 - 1
ambari-server/src/main/java/org/apache/ambari/server/utils/StageUtils.java

@@ -166,7 +166,7 @@ public class StageUtils {
     long now = System.currentTimeMillis();
     s.addHostRoleExecutionCommand(hostname, Role.NAMENODE, RoleCommand.INSTALL,
         new ServiceComponentHostInstallEvent("NAMENODE", hostname, now, "HDP-1.2.0"),
-        "cluster1", "HDFS", false);
+        "cluster1", "HDFS");
     ExecutionCommand execCmd = s.getExecutionCommandWrapper(hostname, "NAMENODE").getExecutionCommand();
     execCmd.setCommandId(s.getActionId());
     List<String> slaveHostList = new ArrayList<String>();

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

@@ -230,7 +230,6 @@ CREATE TABLE stage (
   stage_id BIGINT NOT NULL,
   request_id BIGINT NOT NULL,
   cluster_id BIGINT,
-  skippable SMALLINT DEFAULT 0 NOT NULL,
   log_info VARCHAR(255) NOT NULL,
   request_context VARCHAR(255),
   cluster_host_info LONGBLOB,

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

@@ -221,7 +221,6 @@ CREATE TABLE stage (
   stage_id NUMBER(19) NOT NULL,
   request_id NUMBER(19) NOT NULL,
   cluster_id NUMBER(19) NULL,
-  skippable NUMBER(1) DEFAULT 0 NOT NULL,
   log_info VARCHAR2(255) NULL,
   request_context VARCHAR2(255) NULL,
   cluster_host_info BLOB NOT NULL,

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

@@ -231,7 +231,6 @@ CREATE TABLE stage (
   stage_id BIGINT NOT NULL,
   request_id BIGINT NOT NULL,
   cluster_id BIGINT NOT NULL,
-  skippable SMALLINT DEFAULT 0 NOT NULL,
   log_info VARCHAR(255) NOT NULL,
   request_context VARCHAR(255),
   cluster_host_info BYTEA NOT NULL,

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

@@ -266,7 +266,6 @@ CREATE TABLE ambari.stage (
   stage_id BIGINT NOT NULL,
   request_id BIGINT NOT NULL,
   cluster_id BIGINT NOT NULL,
-  skippable SMALLINT DEFAULT 0 NOT NULL,
   log_info VARCHAR(255) NOT NULL,
   request_context VARCHAR(255),
   cluster_host_info BYTEA NOT NULL,

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

@@ -52,7 +52,7 @@ CREATE TABLE members (member_id INTEGER, group_id INTEGER NOT NULL, user_id INTE
 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, 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, skippable SMALLINT DEFAULT 0 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 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));
 CREATE TABLE requestresourcefilter (filter_id BIGINT NOT NULL, request_id BIGINT NOT NULL, service_name VARCHAR(255), component_name VARCHAR(255), hosts VARBINARY(8000), PRIMARY KEY CLUSTERED (filter_id));
 CREATE TABLE requestoperationlevel (operation_level_id BIGINT NOT NULL, request_id BIGINT NOT NULL, level_name VARCHAR(255), cluster_name VARCHAR(255), service_name VARCHAR(255), host_component_name VARCHAR(255), host_name VARCHAR(255), PRIMARY KEY CLUSTERED (operation_level_id));

+ 1 - 1
ambari-server/src/test/java/org/apache/ambari/server/actionmanager/ExecutionCommandWrapperTest.java

@@ -160,7 +160,7 @@ public class ExecutionCommandWrapperTest {
     s.addHostRoleExecutionCommand(hostName, Role.NAMENODE,
         RoleCommand.START,
         new ServiceComponentHostStartEvent(Role.NAMENODE.toString(),
-            hostName, System.currentTimeMillis()), clusterName, "HDFS", false);
+            hostName, System.currentTimeMillis()), clusterName, "HDFS");
     List<Stage> stages = new ArrayList<Stage>();
     stages.add(s);
     Request request = new Request(stages, clusters);

+ 9 - 9
ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestActionDBAccessorImpl.java

@@ -419,23 +419,23 @@ public class TestActionDBAccessorImpl {
     s.addHostRoleExecutionCommand("host1", Role.HBASE_MASTER,
         RoleCommand.START,
         new ServiceComponentHostStartEvent(Role.HBASE_MASTER.toString(),
-            "host1", System.currentTimeMillis()), "cluster1", "HBASE", false);
+            "host1", System.currentTimeMillis()), "cluster1", "HBASE");
     s.addHostRoleExecutionCommand("host2", Role.HBASE_MASTER,
         RoleCommand.START,
         new ServiceComponentHostStartEvent(Role.HBASE_MASTER.toString(),
-            "host2", System.currentTimeMillis()), "cluster1", "HBASE", false);
+            "host2", System.currentTimeMillis()), "cluster1", "HBASE");
     s.addHostRoleExecutionCommand(
         "host3",
         Role.HBASE_REGIONSERVER,
         RoleCommand.START,
         new ServiceComponentHostStartEvent(Role.HBASE_REGIONSERVER
-            .toString(), "host3", System.currentTimeMillis()), "cluster1", "HBASE", false);
+            .toString(), "host3", System.currentTimeMillis()), "cluster1", "HBASE");
     s.addHostRoleExecutionCommand(
         "host4",
         Role.HBASE_REGIONSERVER,
         RoleCommand.START,
         new ServiceComponentHostStartEvent(Role.HBASE_REGIONSERVER
-            .toString(), "host4", System.currentTimeMillis()), "cluster1", "HBASE", false);
+            .toString(), "host4", System.currentTimeMillis()), "cluster1", "HBASE");
     List<Stage> stages = new ArrayList<Stage>();
     stages.add(s);
     s.getOrderedHostRoleCommands().get(0).setStatus(HostRoleStatus.PENDING);
@@ -502,7 +502,7 @@ public class TestActionDBAccessorImpl {
       clusters.getHost(host).persist();
 
       s.addHostRoleExecutionCommand("host" + i, Role.HBASE_MASTER,
-        RoleCommand.START, null, "cluster1", "HBASE", false);
+        RoleCommand.START, null, "cluster1", "HBASE");
     }
 
     List<Stage> stages = new ArrayList<Stage>();
@@ -546,13 +546,13 @@ public class TestActionDBAccessorImpl {
     s.addHostRoleExecutionCommand(hostname, Role.HBASE_MASTER,
         RoleCommand.START,
         new ServiceComponentHostStartEvent(Role.HBASE_MASTER.toString(),
-            hostname, System.currentTimeMillis()), "cluster1", "HBASE", false);
+            hostname, System.currentTimeMillis()), "cluster1", "HBASE");
     s.addHostRoleExecutionCommand(
         hostname,
         Role.HBASE_REGIONSERVER,
         RoleCommand.START,
         new ServiceComponentHostStartEvent(Role.HBASE_REGIONSERVER
-            .toString(), hostname, System.currentTimeMillis()), "cluster1", "HBASE", false);
+            .toString(), hostname, System.currentTimeMillis()), "cluster1", "HBASE");
     return s;
   }
 
@@ -564,7 +564,7 @@ public class TestActionDBAccessorImpl {
     s.addHostRoleExecutionCommand(hostname, Role.valueOf(actionName),
         RoleCommand.ACTIONEXECUTE,
         new ServiceComponentHostStartEvent(Role.HBASE_MASTER.toString(),
-            hostname, System.currentTimeMillis()), "cluster1", "HBASE", false);
+            hostname, System.currentTimeMillis()), "cluster1", "HBASE");
     List<Stage> stages = new ArrayList<Stage>();
     stages.add(s);
     final RequestResourceFilter resourceFilter = new RequestResourceFilter("HBASE", "HBASE_MASTER", null);
@@ -581,7 +581,7 @@ public class TestActionDBAccessorImpl {
     Stage s = new Stage(requestId, "/a/b", "cluster1", 1L, "action db accessor test",
         "", "commandParamsStage", "hostParamsStage");
     s.setStageId(stageId);
-    s.addServerActionCommand(serverActionName, Role.AMBARI_SERVER_ACTION, RoleCommand.ACTIONEXECUTE, clusterName, null, null, "command details", 300, false);
+    s.addServerActionCommand(serverActionName, Role.AMBARI_SERVER_ACTION, RoleCommand.ACTIONEXECUTE, clusterName, null, null, "command details", 300);
     List<Stage> stages = new ArrayList<Stage>();
     stages.add(s);
     final RequestResourceFilter resourceFilter = new RequestResourceFilter("AMBARI", "SERVER", Arrays.asList(hostname));

+ 1 - 1
ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestActionManager.java

@@ -172,7 +172,7 @@ public class TestActionManager {
     s.addHostRoleExecutionCommand(hostname, Role.HBASE_MASTER,
         RoleCommand.START,
         new ServiceComponentHostStartEvent(Role.HBASE_MASTER.toString(),
-            hostname, System.currentTimeMillis()), "cluster1", "HBASE", false);
+            hostname, System.currentTimeMillis()), "cluster1", "HBASE");
     List<Stage> stages = new ArrayList<Stage>();
     stages.add(s);
     Request request = new Request(stages, clusters);

+ 8 - 8
ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestActionScheduler.java

@@ -222,7 +222,7 @@ public class TestActionScheduler {
       "{\"host_param\":\"param_value\"}", "{\"stage_param\":\"param_value\"}");
     s.addHostRoleExecutionCommand(hostname, Role.SECONDARY_NAMENODE, RoleCommand.INSTALL,
             new ServiceComponentHostInstallEvent("SECONDARY_NAMENODE", hostname, System.currentTimeMillis(), "HDP-1.2.0"),
-            "cluster1", "HDFS", false);
+            "cluster1", "HDFS");
     s.setHostRoleStatus(hostname, "SECONDARY_NAMENODE", HostRoleStatus.IN_PROGRESS);
     stages.add(s);
 
@@ -769,7 +769,7 @@ public class TestActionScheduler {
         RoleCommand.EXECUTE, "cluster1",
         new ServiceComponentHostServerActionEvent(serverHostname, System.currentTimeMillis()),
         payload,
-        null, timeout, false);
+        null, timeout);
 
     return stage;
   }
@@ -1355,7 +1355,7 @@ public class TestActionScheduler {
                                            RoleCommand command, String host, String cluster) {
     stage.addHostRoleExecutionCommand(host, role, command,
         new ServiceComponentHostInstallEvent(role.toString(), host, now, "HDP-0.2"),
-        cluster, service.toString(), false);
+        cluster, service.toString());
     stage.getExecutionCommandWrapper(host,
         role.toString()).getExecutionCommand();
   }
@@ -1384,19 +1384,19 @@ public class TestActionScheduler {
     stage.setStageId(1);
     stage.addHostRoleExecutionCommand("host1", Role.DATANODE, RoleCommand.UPGRADE,
         new ServiceComponentHostUpgradeEvent(Role.DATANODE.toString(), "host1", now, "HDP-0.2"),
-        "cluster1", Service.Type.HDFS.toString(), false);
+        "cluster1", Service.Type.HDFS.toString());
     stage.getExecutionCommandWrapper("host1",
         Role.DATANODE.toString()).getExecutionCommand();
 
     stage.addHostRoleExecutionCommand("host2", Role.DATANODE, RoleCommand.UPGRADE,
         new ServiceComponentHostUpgradeEvent(Role.DATANODE.toString(), "host2", now, "HDP-0.2"),
-        "cluster1", Service.Type.HDFS.toString(), false);
+        "cluster1", Service.Type.HDFS.toString());
     stage.getExecutionCommandWrapper("host2",
         Role.DATANODE.toString()).getExecutionCommand();
 
     stage.addHostRoleExecutionCommand("host3", Role.DATANODE, RoleCommand.UPGRADE,
         new ServiceComponentHostUpgradeEvent(Role.DATANODE.toString(), "host3", now, "HDP-0.2"),
-        "cluster1", Service.Type.HDFS.toString(), false);
+        "cluster1", Service.Type.HDFS.toString());
     stage.getExecutionCommandWrapper("host3",
         Role.DATANODE.toString()).getExecutionCommand();
     stages.add(stage);
@@ -1525,7 +1525,7 @@ public class TestActionScheduler {
     stage.setStageId(stageId);
     stage.addHostRoleExecutionCommand(hostname, role, roleCommand,
         new ServiceComponentHostUpgradeEvent(role.toString(), hostname, System.currentTimeMillis(), "HDP-0.2"),
-        clusterName, service.toString(), false);
+        clusterName, service.toString());
     stage.getExecutionCommandWrapper(hostname,
         role.toString()).getExecutionCommand();
     stage.getOrderedHostRoleCommands().get(0).setTaskId(taskId);
@@ -1552,7 +1552,7 @@ public class TestActionScheduler {
 
     stage.addHostRoleExecutionCommand(hostname, role, roleCommand,
       new ServiceComponentHostInstallEvent(role.toString(), hostname,
-        System.currentTimeMillis(), "HDP-0.2"), clusterName, service.toString(), false);
+        System.currentTimeMillis(), "HDP-0.2"), clusterName, service.toString());
     ExecutionCommand command = stage.getExecutionCommandWrapper
       (hostname, role.toString()).getExecutionCommand();
     command.setTaskId(taskId);

+ 2 - 2
ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestStage.java

@@ -36,9 +36,9 @@ public class TestStage {
   public void testTaskTimeout() {
     Stage s = StageUtils.getATestStage(1, 1, "h1", CLUSTER_HOST_INFO, "{\"host_param\":\"param_value\"}", "{\"stage_param\":\"param_value\"}");
     s.addHostRoleExecutionCommand("h1", Role.DATANODE, RoleCommand.INSTALL,
-        null, "c1", "HDFS", false);
+        null, "c1", "HDFS");
     s.addHostRoleExecutionCommand("h1", Role.HBASE_MASTER, RoleCommand.INSTALL,
-        null, "c1", "HBASE", false);
+        null, "c1", "HBASE");
     for (ExecutionCommandWrapper wrapper : s.getExecutionCommands("h1")) {
       Map<String, String> commandParams = new TreeMap<String, String>();
       commandParams.put(ExecutionCommand.KeyNames.COMMAND_TIMEOUT, "600");

+ 4 - 4
ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatHandler.java

@@ -769,7 +769,7 @@ public class TestHeartbeatHandler {
     s.addHostRoleExecutionCommand(DummyHostname1, Role.HBASE_MASTER,
         RoleCommand.START,
         new ServiceComponentHostStartEvent(Role.HBASE_MASTER.toString(),
-            DummyHostname1, System.currentTimeMillis()), DummyCluster, HBASE, false);
+            DummyHostname1, System.currentTimeMillis()), DummyCluster, HBASE);
     List<Stage> stages = new ArrayList<Stage>();
     stages.add(s);
     Request request = new Request(stages, clusters);
@@ -1154,7 +1154,7 @@ public class TestHeartbeatHandler {
     s.addHostRoleExecutionCommand(DummyHostname1, Role.DATANODE, RoleCommand.INSTALL,
       new ServiceComponentHostInstallEvent(Role.DATANODE.toString(),
         DummyHostname1, System.currentTimeMillis(), "HDP-1.3.0"),
-          DummyCluster, "HDFS", false);
+          DummyCluster, "HDFS");
     List<Stage> stages = new ArrayList<Stage>();
     stages.add(s);
     Request request = new Request(stages, clusters);
@@ -1731,11 +1731,11 @@ public class TestHeartbeatHandler {
     s.addHostRoleExecutionCommand(DummyHostname1, Role.DATANODE, RoleCommand.UPGRADE,
       new ServiceComponentHostUpgradeEvent(Role.DATANODE.toString(),
         DummyHostname1, System.currentTimeMillis(), "HDP-1.3.0"),
-      DummyCluster, "HDFS", false);
+      DummyCluster, "HDFS");
     s.addHostRoleExecutionCommand(DummyHostname1, Role.NAMENODE, RoleCommand.INSTALL,
       new ServiceComponentHostInstallEvent(Role.NAMENODE.toString(),
         DummyHostname1, System.currentTimeMillis(), "HDP-1.3.0"),
-          DummyCluster, "HDFS", false);
+          DummyCluster, "HDFS");
     List<Stage> stages = new ArrayList<Stage>();
     stages.add(s);
     Request request = new Request(stages, clusters);

+ 5 - 5
ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java

@@ -7800,7 +7800,7 @@ public class AmbariManagementControllerTest {
             RoleCommand.START,
             new ServiceComponentHostStartEvent(Role.HBASE_MASTER.toString(),
                     hostName1, System.currentTimeMillis()),
-            clusterName, "HBASE", false);
+            clusterName, "HBASE");
 
     stages.add(new Stage(requestId1, "/a2", clusterName, 1L, context,
       CLUSTER_HOST_INFO, "", ""));
@@ -7808,7 +7808,7 @@ public class AmbariManagementControllerTest {
     stages.get(1).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
             new ServiceComponentHostStartEvent(Role.HBASE_CLIENT.toString(),
-                    hostName1, System.currentTimeMillis()), clusterName, "HBASE", false);
+                    hostName1, System.currentTimeMillis()), clusterName, "HBASE");
 
     stages.add(new Stage(requestId1, "/a3", clusterName, 1L, context,
       CLUSTER_HOST_INFO, "", ""));
@@ -7816,7 +7816,7 @@ public class AmbariManagementControllerTest {
     stages.get(2).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
             new ServiceComponentHostStartEvent(Role.HBASE_CLIENT.toString(),
-                    hostName1, System.currentTimeMillis()), clusterName, "HBASE", false);
+                    hostName1, System.currentTimeMillis()), clusterName, "HBASE");
 
     Request request = new Request(stages, clusters);
     actionDB.persistActions(request);
@@ -7828,7 +7828,7 @@ public class AmbariManagementControllerTest {
     stages.get(0).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
             new ServiceComponentHostStartEvent(Role.HBASE_CLIENT.toString(),
-                    hostName1, System.currentTimeMillis()), clusterName, "HBASE", false);
+                    hostName1, System.currentTimeMillis()), clusterName, "HBASE");
 
     stages.add(new Stage(requestId2, "/a5", clusterName, 1L, context,
       CLUSTER_HOST_INFO, "", ""));
@@ -7836,7 +7836,7 @@ public class AmbariManagementControllerTest {
     stages.get(1).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
             new ServiceComponentHostStartEvent(Role.HBASE_CLIENT.toString(),
-                    hostName1, System.currentTimeMillis()), clusterName, "HBASE", false);
+                    hostName1, System.currentTimeMillis()), clusterName, "HBASE");
 
     request = new Request(stages, clusters);
     actionDB.persistActions(request);

+ 0 - 44
ambari-server/src/test/java/org/apache/ambari/server/controller/internal/StageResourceProviderTest.java

@@ -235,50 +235,6 @@ public class StageResourceProviderTest {
     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));
-  }
-
-
   private List<StageEntity> getStageEntities() {
     StageEntity stage = new StageEntity();
 

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

@@ -99,7 +99,7 @@ public class ServerActionExecutorTest {
         RoleCommand.EXECUTE,
         "cluster1", SERVER_HOST_NAME,
         new ServiceComponentHostServerActionEvent(StageUtils.getHostName(), System.currentTimeMillis()),
-        Collections.<String, String>emptyMap(), null, 1200, false);
+        Collections.<String, String>emptyMap(), null, 1200);
 
     final List<Stage> stages = new ArrayList<Stage>() {
       {
@@ -289,7 +289,7 @@ public class ServerActionExecutorTest {
     stage.addServerActionCommand(MockServerAction.class.getName(), Role.AMBARI_SERVER_ACTION,
         RoleCommand.EXECUTE, "cluster1",
         new ServiceComponentHostServerActionEvent(SERVER_HOST_NAME, System.currentTimeMillis()),
-        payload, "command detail", timeout, false);
+        payload, "command detail", timeout);
 
     return stage;
   }

+ 13 - 13
ambari-server/src/test/java/org/apache/ambari/server/stageplanner/TestStagePlanner.java

@@ -89,10 +89,10 @@ public class TestStagePlanner {
     Stage stage = StageUtils.getATestStage(1, 1, "host1", "", "");
     stage.addHostRoleExecutionCommand("host2", Role.HBASE_MASTER,
         RoleCommand.START, new ServiceComponentHostStartEvent("HBASE_MASTER",
-            "host2", now), "cluster1", "HBASE", false);
+            "host2", now), "cluster1", "HBASE");
     stage.addHostRoleExecutionCommand("host3", Role.ZOOKEEPER_SERVER,
         RoleCommand.START, new ServiceComponentHostStartEvent("ZOOKEEPER_SERVER",
-            "host3", now), "cluster1", "ZOOKEEPER", false);
+            "host3", now), "cluster1", "ZOOKEEPER");
     System.out.println(stage.toString());
 
     rg.build(stage);
@@ -115,37 +115,37 @@ public class TestStagePlanner {
     Stage stage = StageUtils.getATestStage(1, 1, "host1", "", "");
     stage.addHostRoleExecutionCommand("host11", Role.SECONDARY_NAMENODE,
         RoleCommand.START, new ServiceComponentHostStartEvent("SECONDARY_NAMENODE",
-            "host11", now), "cluster1", "HDFS", false);
+            "host11", now), "cluster1", "HDFS");
     stage.addHostRoleExecutionCommand("host2", Role.HBASE_MASTER,
         RoleCommand.START, new ServiceComponentHostStartEvent("HBASE_MASTER",
-            "host2", now), "cluster1", "HBASE", false);
+            "host2", now), "cluster1", "HBASE");
     stage.addHostRoleExecutionCommand("host3", Role.ZOOKEEPER_SERVER,
         RoleCommand.START, new ServiceComponentHostStartEvent("ZOOKEEPER_SERVER",
-            "host3", now), "cluster1", "ZOOKEEPER", false);
+            "host3", now), "cluster1", "ZOOKEEPER");
     stage.addHostRoleExecutionCommand("host4", Role.DATANODE,
         RoleCommand.START, new ServiceComponentHostStartEvent("DATANODE",
-            "host4", now), "cluster1", "HDFS", false);
+            "host4", now), "cluster1", "HDFS");
     stage.addHostRoleExecutionCommand("host4", Role.HBASE_REGIONSERVER,
         RoleCommand.START, new ServiceComponentHostStartEvent("HBASE_REGIONSERVER",
-            "host4", now), "cluster1", "HBASE", false);
+            "host4", now), "cluster1", "HBASE");
     stage.addHostRoleExecutionCommand("host4", Role.TASKTRACKER,
         RoleCommand.START, new ServiceComponentHostStartEvent("TASKTRACKER",
-            "host4", now), "cluster1", "MAPREDUCE", false);
+            "host4", now), "cluster1", "MAPREDUCE");
     stage.addHostRoleExecutionCommand("host5", Role.JOBTRACKER,
         RoleCommand.START, new ServiceComponentHostStartEvent("JOBTRACKER",
-            "host5", now), "cluster1", "MAPREDUCE", false);
+            "host5", now), "cluster1", "MAPREDUCE");
     stage.addHostRoleExecutionCommand("host6", Role.OOZIE_SERVER,
         RoleCommand.START, new ServiceComponentHostStartEvent("OOZIE_SERVER",
-            "host6", now), "cluster1", "OOZIE", false);
+            "host6", now), "cluster1", "OOZIE");
     stage.addHostRoleExecutionCommand("host7", Role.WEBHCAT_SERVER,
         RoleCommand.START, new ServiceComponentHostStartEvent("WEBHCAT_SERVER",
-            "host7", now), "cluster1", "WEBHCAT", false);
+            "host7", now), "cluster1", "WEBHCAT");
     stage.addHostRoleExecutionCommand("host4", Role.GANGLIA_MONITOR,
         RoleCommand.START, new ServiceComponentHostStartEvent("GANGLIA_MONITOR",
-            "host4", now), "cluster1", "GANGLIA", false);
+            "host4", now), "cluster1", "GANGLIA");
     stage.addHostRoleExecutionCommand("host9", Role.GANGLIA_SERVER,
         RoleCommand.START, new ServiceComponentHostStartEvent("GANGLIA_SERVER",
-            "host9", now), "cluster1", "GANGLIA", false);
+            "host9", now), "cluster1", "GANGLIA");
     System.out.println(stage.toString());
     rg.build(stage);
     System.out.println(rg.stringifyGraph());

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

@@ -118,7 +118,6 @@ public class UpgradeCatalog200Test {
     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> stageSkippableColumnCapture = new Capture<DBAccessor.DBColumnInfo>();
 
     Capture<DBAccessor.DBColumnInfo> viewparameterLabelColumnCapture = new Capture<DBAccessor.DBColumnInfo>();
     Capture<DBAccessor.DBColumnInfo> viewparameterPlaceholderColumnCapture = new Capture<DBAccessor.DBColumnInfo>();
@@ -157,10 +156,6 @@ public class UpgradeCatalog200Test {
     dbAccessor.addColumn(eq("host_role_command"),
         capture(hostRoleCommandRetryColumnCapture));
 
-    // Stage skippable
-    dbAccessor.addColumn(eq("stage"),
-        capture(stageSkippableColumnCapture));
-
     // Host Component State: security State
     dbAccessor.addColumn(eq("hostcomponentstate"),
         capture(hostComponentStateSecurityStateColumnCapture));
@@ -235,14 +230,6 @@ public class UpgradeCatalog200Test {
     assertEquals(0, upgradeRetryColumn.getDefaultValue());
     assertFalse(upgradeRetryColumn.isNullable());
 
-    // Verify added column in host_role_command table
-    DBAccessor.DBColumnInfo upgradeSkippableColumn = stageSkippableColumnCapture.getValue();
-    assertEquals("skippable", upgradeSkippableColumn.getName());
-    assertEquals(1, (int) upgradeSkippableColumn.getLength());
-    assertEquals(Integer.class, upgradeSkippableColumn.getType());
-    assertEquals(0, upgradeSkippableColumn.getDefaultValue());
-    assertFalse(upgradeSkippableColumn.isNullable());
-
     // verify security_state columns
     verifyComponentSecurityStateColumn(hostComponentStateSecurityStateColumnCapture);
     verifyComponentSecurityStateColumn(hostComponentDesiredStateSecurityStateColumnCapture);