Kaynağa Gözat

AMBARI-5856. Allow custom actions to be run with api/v1/requests (ncole)

Nate Cole 11 yıl önce
ebeveyn
işleme
4c04bf8720
24 değiştirilmiş dosya ile 540 ekleme ve 239 silme
  1. 3 0
      ambari-agent/src/main/python/ambari_agent/ActionQueue.py
  2. 7 4
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java
  3. 57 41
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionScheduler.java
  4. 17 12
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Request.java
  5. 13 2
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Stage.java
  6. 8 3
      ambari-server/src/main/java/org/apache/ambari/server/actionmanager/StageFactory.java
  7. 3 1
      ambari-server/src/main/java/org/apache/ambari/server/api/query/QueryImpl.java
  8. 4 14
      ambari-server/src/main/java/org/apache/ambari/server/api/resources/RequestResourceDefinition.java
  9. 11 4
      ambari-server/src/main/java/org/apache/ambari/server/api/services/RequestService.java
  10. 75 63
      ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariActionExecutionHelper.java
  11. 36 23
      ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
  12. 29 16
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/RequestResourceProvider.java
  13. 4 1
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/TaskResourceProvider.java
  14. 17 11
      ambari-server/src/main/java/org/apache/ambari/server/stageplanner/RoleGraph.java
  15. 1 1
      ambari-server/src/main/java/org/apache/ambari/server/utils/StageUtils.java
  16. 1 1
      ambari-server/src/main/resources/custom_action_definitions/system_action_definitions.xml
  17. 1 1
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/ExecutionCommandWrapperTest.java
  18. 3 3
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestActionDBAccessorImpl.java
  19. 1 1
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestActionManager.java
  20. 10 6
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestActionScheduler.java
  21. 1 1
      ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestStage.java
  22. 3 3
      ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatHandler.java
  23. 98 6
      ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java
  24. 137 21
      ambari-server/src/test/java/org/apache/ambari/server/controller/internal/RequestResourceProviderTest.java

+ 3 - 0
ambari-agent/src/main/python/ambari_agent/ActionQueue.py

@@ -98,6 +98,9 @@ class ActionQueue(threading.Thread):
     for command in commands:
       if not command.has_key('serviceName'):
         command['serviceName'] = "null"
+      if not command.has_key('clusterName'):
+        command['clusterName'] = 'null'
+
       logger.info("Adding " + command['commandType'] + " for service " + \
                   command['serviceName'] + " of cluster " + \
                   command['clusterName'] + " to the queue.")

+ 7 - 4
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java

@@ -152,6 +152,7 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
       if (command.getStatus() == HostRoleStatus.QUEUED ||
           command.getStatus() == HostRoleStatus.IN_PROGRESS ||
           command.getStatus() == HostRoleStatus.PENDING) {
+
         command.setStatus(HostRoleStatus.ABORTED);
         command.setEndTime(now);
         LOG.info("Aborting command. Hostname " + command.getHostName()
@@ -203,11 +204,13 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
 
     RequestEntity requestEntity = request.constructNewPersistenceEntity();
 
+    Long clusterId = Long.valueOf(-1L);
     ClusterEntity clusterEntity = clusterDAO.findById(request.getClusterId());
-    if (clusterEntity == null) {
-      throw new RuntimeException(String.format("Cluster with id=%s not found", request.getClusterId()));
+    if (clusterEntity != null) {
+      clusterId = clusterEntity.getClusterId();
     }
-    requestEntity.setClusterId(clusterEntity.getClusterId());
+    
+    requestEntity.setClusterId(clusterId);
     requestDAO.create(requestEntity);
 
     //TODO wire request to cluster
@@ -216,7 +219,7 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
     for (Stage stage : request.getStages()) {
       StageEntity stageEntity = stage.constructNewPersistenceEntity();
       stageEntities.add(stageEntity);
-      stageEntity.setClusterId(clusterEntity.getClusterId());
+      stageEntity.setClusterId(clusterId);
       //TODO refactor to reduce merges
       stageEntity.setRequest(requestEntity);
       stageDAO.create(stageEntity);

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

@@ -375,65 +375,80 @@ class ActionScheduler implements Runnable {
     if (taskTimeoutAdjustment) {
       taskTimeout = actionTimeout + s.getStageTimeout();
     }
+
+    Cluster cluster = null;
+    if (null != s.getClusterName()) {
+      cluster = fsmObject.getCluster(s.getClusterName());
+    }
+
     for (String host : s.getHosts()) {
       List<ExecutionCommandWrapper> commandWrappers = s.getExecutionCommands(host);
-      Cluster cluster = fsmObject.getCluster(s.getClusterName());
       Host hostObj = fsmObject.getHost(host);
+
       for(ExecutionCommandWrapper wrapper : commandWrappers) {
         ExecutionCommand c = wrapper.getExecutionCommand();
         String roleStr = c.getRole();
         HostRoleStatus status = s.getHostRoleStatus(host, roleStr);
-        Service svc = null;
-        if (c.getServiceName() != null && !c.getServiceName().isEmpty()) {
-          svc = cluster.getService(c.getServiceName());
-        }
-        ServiceComponent svcComp = null;
-        Map<String, ServiceComponentHost>  scHosts = null;
-        try {
-          if (svc != null) {
-            svcComp = svc.getServiceComponent(roleStr);
-            scHosts = svcComp.getServiceComponentHosts();
+
+        boolean hostDeleted = false;
+        if (null != cluster) {
+          Service svc = null;
+          if (c.getServiceName() != null && !c.getServiceName().isEmpty()) {
+            svc = cluster.getService(c.getServiceName());
+          }
+
+          ServiceComponent svcComp = null;
+          Map<String, ServiceComponentHost> scHosts = null;
+          try {
+            if (svc != null) {
+              svcComp = svc.getServiceComponent(roleStr);
+              scHosts = svcComp.getServiceComponentHosts();
+            }
+          } catch (ServiceComponentNotFoundException scnex) {
+            String msg = String.format(
+                    "%s is not not a service component, assuming its an action",
+                    roleStr);
+            LOG.debug(msg);
+          }
+
+          hostDeleted = (scHosts != null && !scHosts.containsKey(host));
+          if (hostDeleted) {
+            String message = String.format(
+              "Host component information has not been found.  Details:" +
+              "cluster=%s; host=%s; service=%s; component=%s; ",
+              c.getClusterName(), host,
+              svcComp == null ? "null" : svcComp.getServiceName(),
+              svcComp == null ? "null" : svcComp.getName());
+            LOG.warn(message);
           }
-        } catch (ServiceComponentNotFoundException scnex) {
-          String msg = String.format(
-                  "%s is not not a service component, assuming its an action",
-                  roleStr);
-          LOG.debug(msg);
         }
+
         // Check that service host component is not deleted
-        if (scHosts!= null && ! scHosts.containsKey(host)) {
+        if (hostDeleted) {
+          
           String message = String.format(
-                  "Service component host not found when trying to " +
-                          "schedule an execution command. " +
-                          "The most probable reason " +
-                          "for that is that host component " +
-                          "has been deleted recently. "+
-                          "The command has been aborted and dequeued. " +
-                          "Execution command details: " +
-                          "cluster=%s; host=%s; service=%s; component=%s; " +
-                          "cmdId: %s; taskId: %s; roleCommand: %s",
-                  c.getClusterName(), host, svcComp == null ? "null" : svcComp.getServiceName(),
-                  svcComp == null ? "null" : svcComp.getName(),
-                  c.getCommandId(), c.getTaskId(), c.getRoleCommand());
-          LOG.warn(message);
+            "Host not found when trying to schedule an execution command. " +
+            "The most probable reason for that is that host or host component " +
+            "has been deleted recently. The command has been aborted and dequeued." +
+            "Execution command details: " + 
+            "cmdId: %s; taskId: %s; roleCommand: %s",
+            c.getCommandId(), c.getTaskId(), c.getRoleCommand());
+          LOG.warn("Host {} has been detected as non-available. {}", host, message);
           // Abort the command itself
-          db.abortHostRole(host, s.getRequestId(),
-                  s.getStageId(), c.getRole(), message);
+          db.abortHostRole(host, s.getRequestId(), s.getStageId(), c.getRole(), message);
           status = HostRoleStatus.ABORTED;
-        } else if (timeOutActionNeeded(status, s, hostObj, roleStr, now,
-                taskTimeout)) {
+        } else if (timeOutActionNeeded(status, s, hostObj, roleStr, now, taskTimeout)) {
           // Process command timeouts
-          LOG.info("Host:" + host + ", role:" + roleStr + ", actionId:"
-                  + s.getActionId() + " timed out");
+          LOG.info("Host:" + host + ", role:" + roleStr + ", actionId:" + s.getActionId() + " timed out");
           if (s.getAttemptCount(host, roleStr) >= maxAttempts) {
-            LOG.warn("Host:" + host + ", role:" + roleStr + ", actionId:"
-                + s.getActionId() + " expired");
+            LOG.warn("Host:" + host + ", role:" + roleStr + ", actionId:" + s.getActionId() + " expired");
             db.timeoutHostRole(host, s.getRequestId(), s.getStageId(), c.getRole());
             //Reinitialize status
             status = s.getHostRoleStatus(host, roleStr);
-            transitionToFailedState(cluster.getClusterName(),
-              c.getServiceName(), roleStr, host, now, false);
-            LOG.warn("Operation timed out. Role: " + roleStr + ", host: " + host);
+
+            if (null != cluster) {
+              transitionToFailedState(cluster.getClusterName(), c.getServiceName(), roleStr, host, now, false);
+            }
 
             // Dequeue command
             actionQueue.dequeue(host, c.getCommandId());
@@ -445,6 +460,7 @@ class ActionScheduler implements Runnable {
           //Need to schedule first time
           commandsToSchedule.add(c);
         }
+
         this.updateRoleStats(status, roleStats.get(roleStr));
       }
     }

+ 17 - 12
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Request.java

@@ -67,16 +67,19 @@ public class Request {
    */
   public Request(@Assisted long requestId, @Assisted("clusterId") Long clusterId, Clusters clusters) {
     this.requestId = requestId;
-    this.clusterId = clusterId;
+    this.clusterId = clusterId.longValue();
     this.createTime = System.currentTimeMillis();
     this.startTime = -1;
     this.endTime = -1;
-    try {
-      this.clusterName = clusters.getClusterById(clusterId).getClusterName();
-    } catch (AmbariException e) {
-      String message = String.format("Cluster with id=%s not found", clusterId);
-      LOG.error(message);
-      throw new RuntimeException(message);
+
+    if (-1L != this.clusterId) {
+      try {
+        this.clusterName = clusters.getClusterById(this.clusterId).getClusterName();
+      } catch (AmbariException e) {
+        String message = String.format("Cluster with id=%s not found", clusterId);
+        LOG.error(message);
+        throw new RuntimeException(message);
+      }
     }
   }
 
@@ -93,10 +96,12 @@ public class Request {
       this.clusterName = stage.getClusterName();
       try {
         this.clusterId = clusters.getCluster(clusterName).getClusterId();
-      } catch (AmbariException e) {
-        String message = String.format("Cluster %s not found", clusterName);
-        LOG.error(message);
-        throw new RuntimeException(message);
+      } catch (Exception e) {
+        if (null != clusterName) {
+          String message = String.format("Cluster %s not found", clusterName);
+          LOG.error(message);
+          throw new RuntimeException(message);
+        }
       }
       this.requestContext = stages.iterator().next().getRequestContext();
       this.createTime = System.currentTimeMillis();
@@ -261,7 +266,7 @@ public class Request {
 
 
   public Long getClusterId() {
-    return clusterId;
+    return Long.valueOf(clusterId);
   }
 
   public String getClusterName() {

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

@@ -54,6 +54,7 @@ public class Stage {
   private static Logger LOG = LoggerFactory.getLogger(Stage.class);
   private final long requestId;
   private String clusterName;
+  private long clusterId = -1L;
   private long stageId = -1;
   private final String logDir;
   private final String requestContext;
@@ -73,12 +74,17 @@ public class Stage {
       new TreeMap<String, List<ExecutionCommandWrapper>>();
 
   @AssistedInject
-  public Stage(@Assisted long requestId, @Assisted("logDir") String logDir, @Assisted("clusterName") String clusterName,
-               @Assisted("requestContext") @Nullable String requestContext, @Assisted("clusterHostInfo") String clusterHostInfo) {
+  public Stage(@Assisted long requestId,
+      @Assisted("logDir") String logDir,
+      @Assisted("clusterName") @Nullable String clusterName,
+      @Assisted("clusterId") long clusterId,
+      @Assisted("requestContext") @Nullable String requestContext,
+      @Assisted("clusterHostInfo") String clusterHostInfo) {
     this.wrappersLoaded = true;
     this.requestId = requestId;
     this.logDir = logDir;
     this.clusterName = clusterName;
+    this.clusterId = clusterId;
     this.requestContext = requestContext == null ? "" : requestContext;
     this.clusterHostInfo = clusterHostInfo;
   }
@@ -349,6 +355,11 @@ public class Stage {
   public String getClusterName() {
     return clusterName;
   }
+  
+  public long getClusterId() {
+    return clusterId;
+  }
+  
 
   public String getRequestContext() {
     return requestContext;

+ 8 - 3
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/StageFactory.java

@@ -18,13 +18,18 @@
 
 package org.apache.ambari.server.actionmanager;
 
-import com.google.inject.assistedinject.Assisted;
 import org.apache.ambari.server.orm.entities.StageEntity;
 
+import com.google.inject.assistedinject.Assisted;
+
 public interface StageFactory {
 
-  Stage createNew(long requestId, @Assisted("logDir") String logDir, @Assisted("clusterName") String clusterName,
-                   @Assisted("requestContext") String requestContext, @Assisted("clusterHostInfo") String clusterHostInfo);
+  Stage createNew(long requestId,
+      @Assisted("logDir") String logDir,
+      @Assisted("clusterName") String clusterName,
+      @Assisted("clusterId") long clusterId,
+      @Assisted("requestContext") String requestContext,
+      @Assisted("clusterHostInfo") String clusterHostInfo);
 
   Stage createExisting(StageEntity stageEntity);
 }

+ 3 - 1
ambari-server/src/main/java/org/apache/ambari/server/api/query/QueryImpl.java

@@ -887,7 +887,9 @@ public class QueryImpl implements Query, ResourceInstance {
 
     for (Resource.Type type : types) {
       String resourceKeyProp = schema.getKeyPropertyId(type);
-      resourceKeyValueMap.put(type, resource.getPropertyValue(resourceKeyProp).toString());
+      Object resourceValue = resource.getPropertyValue(resourceKeyProp);
+      if (null != resourceValue)
+        resourceKeyValueMap.put(type, resourceValue.toString());
     }
     return resourceKeyValueMap;
   }

+ 4 - 14
ambari-server/src/main/java/org/apache/ambari/server/api/resources/RequestResourceDefinition.java

@@ -63,30 +63,20 @@ public class RequestResourceDefinition extends BaseResourceDefinition {
   private class RequestHrefPostProcessor implements PostProcessor {
     @Override
     public void process(Request request, TreeNode<Resource> resultNode, String href) {
-      StringBuilder sb = new StringBuilder();
-      String[] toks = href.split("/");
-
-      for (int i = 0; i < toks.length; ++i) {
-        String s = toks[i];
-        sb.append(s).append('/');
-        if ("clusters".equals(s)) {
-          sb.append(toks[i + 1]).append('/');
-          break;
-        }
-      }
 
       Object requestId = resultNode.getObject().getPropertyValue(getClusterController().
           getSchema(Resource.Type.Request).getKeyPropertyId(Resource.Type.Request));
 
-      sb.append("requests/").append(requestId);
-
+      StringBuilder sb = new StringBuilder(href);
+      if (href.endsWith("/requests"))
+        sb.append('/').append(requestId);
+        
       resultNode.setProperty("href", sb.toString());
     }
   }
 
   private class RequestSourceScheduleHrefPostProcessor implements PostProcessor {
 
-    @SuppressWarnings("unchecked")
     @Override
     public void process(Request request, TreeNode<Resource> resultNode, String href) {
       StringBuilder sb = new StringBuilder();

+ 11 - 4
ambari-server/src/main/java/org/apache/ambari/server/api/services/RequestService.java

@@ -38,6 +38,7 @@ import java.util.Map;
 /**
  * Service responsible for request resource requests.
  */
+@Path("/requests/")
 public class RequestService extends BaseService {
   /**
    * Parent cluster name.
@@ -45,6 +46,9 @@ public class RequestService extends BaseService {
   private String m_clusterName;
 
 
+  public RequestService() {
+  }
+  
   /**
    * Constructor.
    *
@@ -55,7 +59,8 @@ public class RequestService extends BaseService {
   }
 
   /**
-   * Handles URL: /clusters/{clusterID}/requests/{requestID}
+   * Handles URL: /clusters/{clusterID}/requests/{requestID} or
+   * /requests/{requestId}
    * Get a specific request.
    *
    * @param headers    http headers
@@ -75,7 +80,7 @@ public class RequestService extends BaseService {
   }
 
   /**
-   * Handles URL: /clusters/{clusterId}/requests
+   * Handles URL: /clusters/{clusterId}/requests or /requests
    * Get all requests for a cluster.
    *
    * @param headers http headers
@@ -99,7 +104,7 @@ public class RequestService extends BaseService {
   }
 
   /**
-   * Handles: POST /clusters/{clusterId}/requests
+   * Handles: POST /clusters/{clusterId}/requests or /requests
    * Create multiple services.
    *
    * @param body        http body
@@ -124,7 +129,9 @@ public class RequestService extends BaseService {
    */
   ResourceInstance createRequestResource(String clusterName, String requestId) {
     Map<Resource.Type,String> mapIds = new HashMap<Resource.Type, String>();
-    mapIds.put(Resource.Type.Cluster, clusterName);
+    
+    if (null != clusterName)
+      mapIds.put(Resource.Type.Cluster, clusterName);
     mapIds.put(Resource.Type.Request, requestId);
 
     return createResource(Resource.Type.Request, mapIds);

+ 75 - 63
ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariActionExecutionHelper.java

@@ -18,8 +18,17 @@
 
 package org.apache.ambari.server.controller;
 
-import com.google.inject.Inject;
-import com.google.inject.Singleton;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.COMMAND_TIMEOUT;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.COMPONENT_CATEGORY;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.SCRIPT;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.SCRIPT_TYPE;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.Role;
 import org.apache.ambari.server.RoleCommand;
@@ -43,16 +52,8 @@ import org.apache.ambari.server.utils.StageUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.COMMAND_TIMEOUT;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.COMPONENT_CATEGORY;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.SCRIPT;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.SCRIPT_TYPE;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
 
 /**
  * Helper class containing logic to process custom action execution requests
@@ -63,15 +64,11 @@ public class AmbariActionExecutionHelper {
       LoggerFactory.getLogger(AmbariActionExecutionHelper.class);
   private static final String TYPE_PYTHON = "PYTHON";
 
-  @Inject
-  private ActionMetadata actionMetadata;
   @Inject
   private Clusters clusters;
   @Inject
   private AmbariManagementController managementController;
   @Inject
-  private ActionManager actionManager;
-  @Inject
   private AmbariMetaInfo ambariMetaInfo;
   @Inject
   private MaintenanceStateHelper maintenanceStateHelper;
@@ -82,13 +79,9 @@ public class AmbariActionExecutionHelper {
    * @throws AmbariException
    */
   public void validateAction(ExecuteActionRequest actionRequest) throws AmbariException {
-    Cluster cluster = clusters.getCluster(actionRequest.getClusterName());
-
-    if (cluster == null) {
-      throw new AmbariException("Unable to find cluster. clusterName = " +
-        actionRequest.getClusterName());
-    }
+    
 
+    
     if (actionRequest.getActionName() == null || actionRequest.getActionName().isEmpty()) {
       throw new AmbariException("Action name must be specified");
     }
@@ -115,33 +108,43 @@ public class AmbariActionExecutionHelper {
       }
     }
 
-
-    StackId stackId = cluster.getCurrentStackVersion();
-    String expectedService = actionDef.getTargetService() == null ? "" : actionDef.getTargetService();
-
     List<RequestResourceFilter> resourceFilters = actionRequest.getResourceFilters();
-    String targetService = "";
-    String targetComponent = "";
     RequestResourceFilter resourceFilter = null;
-
     if (resourceFilters != null && !resourceFilters.isEmpty()) {
       if (resourceFilters.size() > 1) {
         throw new AmbariException("Custom action definition only allows one " +
           "resource filter to be specified.");
+      } else {
+        resourceFilter = resourceFilters.get(0);
       }
+    }
 
-      resourceFilter = resourceFilters.get(0);
-      String actualService = resourceFilter.getServiceName() == null ? "" : resourceFilter.getServiceName();
+    String targetService = "";
+    String targetComponent = "";
+    
+    if (null != actionRequest.getClusterName()) {
+      Cluster cluster = clusters.getCluster(actionRequest.getClusterName());
+      
+      if (cluster == null) {
+        throw new AmbariException("Unable to find cluster. clusterName = " +
+          actionRequest.getClusterName());
+      }
+      
+      StackId stackId = cluster.getCurrentStackVersion();
+
+      String expectedService = actionDef.getTargetService() == null ? "" : actionDef.getTargetService();
+  
+      String actualService = resourceFilter == null || resourceFilter.getServiceName() == null ? "" : resourceFilter.getServiceName();
       if (!expectedService.isEmpty() && !actualService.isEmpty() && !expectedService.equals(actualService)) {
         throw new AmbariException("Action " + actionRequest.getActionName() + " targets service " + actualService +
           " that does not match with expected " + expectedService);
       }
-
+  
       targetService = expectedService;
       if (targetService == null || targetService.isEmpty()) {
         targetService = actualService;
       }
-
+  
       if (targetService != null && !targetService.isEmpty()) {
         ServiceInfo serviceInfo;
         try {
@@ -156,14 +159,14 @@ public class AmbariActionExecutionHelper {
             " targets service " + targetService + " that does not exist.");
         }
       }
-
+  
       String expectedComponent = actionDef.getTargetComponent() == null ? "" : actionDef.getTargetComponent();
-      String actualComponent = resourceFilter.getComponentName() == null ? "" : resourceFilter.getComponentName();
+      String actualComponent = resourceFilter == null || resourceFilter.getComponentName() == null ? "" : resourceFilter.getComponentName();
       if (!expectedComponent.isEmpty() && !actualComponent.isEmpty() && !expectedComponent.equals(actualComponent)) {
         throw new AmbariException("Action " + actionRequest.getActionName() + " targets component " + actualComponent +
           " that does not match with expected " + expectedComponent);
       }
-
+  
       targetComponent = expectedComponent;
       if (targetComponent == null || targetComponent.isEmpty()) {
         targetComponent = actualComponent;
@@ -173,7 +176,7 @@ public class AmbariActionExecutionHelper {
         throw new AmbariException("Action " + actionRequest.getActionName() + " targets component " + targetComponent +
           " without specifying the target service.");
       }
-
+  
       if (targetComponent != null && !targetComponent.isEmpty()) {
         ComponentInfo compInfo;
         try {
@@ -189,7 +192,7 @@ public class AmbariActionExecutionHelper {
         }
       }
     }
-
+        
     if (TargetHostType.SPECIFIC.equals(actionDef.getTargetType())
       || (targetService.isEmpty() && targetComponent.isEmpty())) {
       if (resourceFilter == null || resourceFilter.getHostNames().size() == 0) {
@@ -213,7 +216,11 @@ public class AmbariActionExecutionHelper {
 
     String actionName = actionContext.getActionName();
     String clusterName = actionContext.getClusterName();
-    Cluster cluster = clusters.getCluster(clusterName);
+    Cluster cluster = null;
+    if (null != clusterName)
+      cluster = clusters.getCluster(clusterName);
+    
+    ComponentInfo componentInfo = null;
 
     List<RequestResourceFilter> resourceFilters = actionContext.getResourceFilters();
 
@@ -227,36 +234,38 @@ public class AmbariActionExecutionHelper {
 
     String serviceName = actionContext.getExpectedServiceName();
     String componentName = actionContext.getExpectedComponentName();
-    StackId stackId = cluster.getCurrentStackVersion();
-    ComponentInfo componentInfo = null;
-
-    if (serviceName != null && !serviceName.isEmpty()) {
-      if (componentName != null && !componentName.isEmpty()) {
-        Map<String, ServiceComponentHost> componentHosts =
-          cluster.getService(serviceName)
-            .getServiceComponent(componentName).getServiceComponentHosts();
-        candidateHosts.addAll(componentHosts.keySet());
-        componentInfo = ambariMetaInfo.getComponentCategory(stackId.getStackName(),
-          stackId.getStackVersion(), serviceName, componentName);
-      } else {
-        for (String component : cluster.getService(serviceName).getServiceComponents().keySet()) {
+    
+    if (null != cluster) {
+      StackId stackId = cluster.getCurrentStackVersion();
+      if (serviceName != null && !serviceName.isEmpty()) {
+        if (componentName != null && !componentName.isEmpty()) {
           Map<String, ServiceComponentHost> componentHosts =
             cluster.getService(serviceName)
-              .getServiceComponent(component).getServiceComponentHosts();
+              .getServiceComponent(componentName).getServiceComponentHosts();
           candidateHosts.addAll(componentHosts.keySet());
+          componentInfo = ambariMetaInfo.getComponentCategory(stackId.getStackName(),
+            stackId.getStackVersion(), serviceName, componentName);
+        } else {
+          for (String component : cluster.getService(serviceName).getServiceComponents().keySet()) {
+            Map<String, ServiceComponentHost> componentHosts =
+              cluster.getService(serviceName)
+                .getServiceComponent(component).getServiceComponentHosts();
+            candidateHosts.addAll(componentHosts.keySet());
+          }
         }
+      } else {
+        // All hosts are valid target host
+        candidateHosts.addAll(clusters.getHostsForCluster(cluster.getClusterName()).keySet());
       }
-    } else {
-      // All hosts are valid target host
-      candidateHosts.addAll(clusters.getHostsForCluster(cluster.getClusterName()).keySet());
     }
 
     // If request did not specify hosts and there exists no host
     if (resourceFilter.getHostNames().isEmpty() && candidateHosts.isEmpty()) {
       throw new AmbariException("Suitable hosts not found, component="
         + componentName + ", service=" + serviceName
-        + ", cluster=" + cluster.getClusterName() + ", " +
-        "actionName=" + actionContext.getActionName());
+        + ((null == cluster) ? "" : ", cluster=" + cluster.getClusterName() + ", ")
+//        + ", cluster=" + cluster.getClusterName() + ", "
+        + "actionName=" + actionContext.getActionName());
     }
 
     // Compare specified hosts to available hosts
@@ -294,8 +303,10 @@ public class AmbariActionExecutionHelper {
       }
     }
 
-    Set<Map<String, String>> maintenanceSCHs =
-      maintenanceStateHelper.getMaintenanceHostComponents(clusters, cluster);
+    Set<Map<String, String>> maintenanceSCHs = new HashSet<Map<String, String>>();
+        
+    if (null != cluster)
+      maintenanceSCHs = maintenanceStateHelper.getMaintenanceHostComponents(clusters, cluster);
 
     //create tasks for each host
     for (String hostName : targetHosts) {
@@ -307,7 +318,7 @@ public class AmbariActionExecutionHelper {
 
       Map<String, Map<String, String>> configurations = new TreeMap<String, Map<String, String>>();
       Map<String, Map<String, String>> configTags = null;
-      if (!serviceName.isEmpty()) {
+      if (!serviceName.isEmpty() && null != cluster) {
         configTags = managementController.findConfigurationTagsWithOverrides(cluster, hostName);
       }
 
@@ -343,8 +354,9 @@ public class AmbariActionExecutionHelper {
       execCmd.setRoleParams(roleParams);
 
       // Generate cluster host info
-      execCmd.setClusterHostInfo(
-        StageUtils.getClusterHostInfo(clusters.getHostsForCluster(clusterName), cluster));
+      if (null != cluster)
+        execCmd.setClusterHostInfo(
+          StageUtils.getClusterHostInfo(clusters.getHostsForCluster(clusterName), cluster));
 
       // cluster passive map
       execCmd.setPassiveInfo(maintenanceSCHs);

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

@@ -654,7 +654,10 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
   private Stage createNewStage(long id, Cluster cluster, long requestId, String requestContext, String clusterHostInfo) {
     String logDir = BASE_LOG_DIR + File.pathSeparator + requestId;
     Stage stage =
-        stageFactory.createNew(requestId, logDir, cluster.getClusterName(), requestContext, clusterHostInfo);
+        stageFactory.createNew(requestId, logDir,
+            null == cluster ? null : cluster.getClusterName(),
+            null == cluster ? -1L : cluster.getClusterId(),
+            requestContext, clusterHostInfo);
     stage.setStageId(id);
     return stage;
   }
@@ -2541,7 +2544,8 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
   public RequestStatusResponse createAction(ExecuteActionRequest actionRequest,
       Map<String, String> requestProperties)
       throws AmbariException {
-    String clusterName;
+    String clusterName = actionRequest.getClusterName();
+        
     String requestContext = "";
 
     if (requestProperties != null) {
@@ -2552,44 +2556,53 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
       }
     }
 
-    if (actionRequest.getClusterName() == null
-        || actionRequest.getClusterName().isEmpty()) {
-      throw new AmbariException("Invalid request, cluster name must be specified");
-    }
-    clusterName = actionRequest.getClusterName();
-
-    Cluster cluster = clusters.getCluster(clusterName);
-
-    LOG.info("Received action execution request"
-      + ", clusterName=" + actionRequest.getClusterName()
-      + ", request=" + actionRequest.toString());
+    Cluster cluster = null;
+    if (null != clusterName) {
+      cluster = clusters.getCluster(clusterName);
 
+      LOG.info("Received action execution request"
+        + ", clusterName=" + actionRequest.getClusterName()
+        + ", request=" + actionRequest.toString());
+    }
+    
     ActionExecutionContext actionExecContext = getActionExecutionContext(actionRequest);
     if (actionRequest.isCommand()) {
       customCommandExecutionHelper.validateAction(actionRequest);
     } else {
       actionExecutionHelper.validateAction(actionRequest);
     }
-
-    Map<String, Set<String>> clusterHostInfo = StageUtils.getClusterHostInfo(
+    
+    Map<String, String> params = new HashMap<String, String>();
+    Map<String, Set<String>> clusterHostInfo = new HashMap<String, Set<String>>();
+    String clusterHostInfoJson = "{}";
+    
+    if (null != cluster) {
+      clusterHostInfo = StageUtils.getClusterHostInfo(
         clusters.getHostsForCluster(cluster.getClusterName()), cluster);
-
-    String clusterHostInfoJson = StageUtils.getGson().toJson(clusterHostInfo);
+      params = createDefaultHostParams(cluster);
+      clusterHostInfoJson = StageUtils.getGson().toJson(clusterHostInfo);
+    }
+    
     Stage stage = createNewStage(0, cluster, actionManager.getNextRequestId(), requestContext, clusterHostInfoJson);
 
-    Map<String, String> params = createDefaultHostParams(cluster);
-
     if (actionRequest.isCommand()) {
       customCommandExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage,
           params, requestProperties);
     } else {
       actionExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage, params);
     }
-
-    RoleCommandOrder rco = this.getRoleCommandOrder(cluster);
-    RoleGraph rg = new RoleGraph(rco);
-    rg.build(stage);
+    
+    RoleGraph rg = null;
+    if (null != cluster) {
+      RoleCommandOrder rco = getRoleCommandOrder(cluster);
+      rg = new RoleGraph(rco);
+    } else {
+      rg = new RoleGraph();
+    }
+    
+    rg.build(stage);    
     List<Stage> stages = rg.getStages();
+    
     if (stages != null && !stages.isEmpty()) {
       actionManager.sendActions(stages, actionRequest);
       return getRequestStatusResponse(stage.getRequestId());

+ 29 - 16
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/RequestResourceProvider.java

@@ -35,6 +35,7 @@ import org.apache.ambari.server.controller.spi.ResourceAlreadyExistsException;
 import org.apache.ambari.server.controller.spi.SystemException;
 import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
 import org.apache.ambari.server.state.Clusters;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -138,22 +139,30 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
     Integer maxResults = (maxResultsRaw == null ? null : Integer.parseInt(maxResultsRaw));
     Boolean ascOrder = (ascOrderRaw == null ? null : Boolean.parseBoolean(ascOrderRaw));
 
-    for (Map<String, Object> properties : getPropertyMaps(predicate)) {
-      String clusterName = (String) properties.get(REQUEST_CLUSTER_NAME_PROPERTY_ID);
-
-      Long requestId = null;
-      if (properties.get(REQUEST_ID_PROPERTY_ID) != null) {
-        requestId = Long.valueOf((String) properties.get(REQUEST_ID_PROPERTY_ID));
-      }
-
-      String requestStatus = null;
-      if (properties.get(REQUEST_STATUS_PROPERTY_ID) != null) {
-        requestStatus = (String) properties.get(REQUEST_STATUS_PROPERTY_ID);
+    Set<Map<String, Object>> propertyMaps = new HashSet<Map<String,Object>>();
+    
+    if (null == predicate) {
+      resources.addAll(
+          getRequestResources(null, null, null, maxResults, ascOrder, requestedIds));
+    } else {
+      for (Map<String, Object> properties : getPropertyMaps(predicate)) {
+        String clusterName = (String) properties.get(REQUEST_CLUSTER_NAME_PROPERTY_ID);
+  
+        Long requestId = null;
+        if (properties.get(REQUEST_ID_PROPERTY_ID) != null) {
+          requestId = Long.valueOf((String) properties.get(REQUEST_ID_PROPERTY_ID));
+        }
+  
+        String requestStatus = null;
+        if (properties.get(REQUEST_STATUS_PROPERTY_ID) != null) {
+          requestStatus = (String) properties.get(REQUEST_STATUS_PROPERTY_ID);
+        }
+  
+        resources.addAll(getRequestResources(clusterName, requestId, requestStatus, maxResults,
+            ascOrder, requestedIds));
       }
-
-      resources.addAll(getRequestResources(clusterName, requestId, requestStatus, maxResults,
-          ascOrder, requestedIds));
     }
+    
     return resources;
   }
 
@@ -335,7 +344,9 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
     Map<Long, Resource> resourceMap = new HashMap<Long, Resource>();
 
     for (org.apache.ambari.server.actionmanager.Request request : requests) {
-      resourceMap.put(request.getRequestId(), getRequestResource(request, requestedPropertyIds));
+      if ((null == clusterName && null == request.getClusterName()) ||
+          (null != clusterName && null != request.getClusterName() && clusterName.equals(request.getClusterName())))
+        resourceMap.put(request.getRequestId(), getRequestResource(request, requestedPropertyIds));
     }
 
     return resourceMap.values();
@@ -345,7 +356,9 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
                                       Set<String> requestedPropertyIds) {
     Resource resource = new ResourceImpl(Resource.Type.Request);
 
-    setResourceProperty(resource, REQUEST_CLUSTER_NAME_PROPERTY_ID, request.getClusterName(), requestedPropertyIds);
+    if (null != request.getClusterName())
+      setResourceProperty(resource, REQUEST_CLUSTER_NAME_PROPERTY_ID, request.getClusterName(), requestedPropertyIds);
+
     setResourceProperty(resource, REQUEST_ID_PROPERTY_ID, request.getRequestId(), requestedPropertyIds);
     setResourceProperty(resource, REQUEST_CONTEXT_ID, request.getRequestContext(), requestedPropertyIds);
     setResourceProperty(resource, REQUEST_TYPE_ID, request.getRequestType(), requestedPropertyIds);

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

@@ -134,7 +134,10 @@ class TaskResourceProvider extends AbstractControllerResourceProvider {
       for (TaskStatusResponse response : responses) {
         Resource resource = new ResourceImpl(Resource.Type.Task);
 
-        setResourceProperty(resource, TASK_CLUSTER_NAME_PROPERTY_ID, entry.getKey(), requestedIds);
+        // !!! shocked this isn't broken.  the key can be null for non-cluster tasks
+        if (null != entry.getKey())
+          setResourceProperty(resource, TASK_CLUSTER_NAME_PROPERTY_ID, entry.getKey(), requestedIds);  
+        
         setResourceProperty(resource, TASK_REQUEST_ID_PROPERTY_ID, response.getRequestId(), requestedIds);
         setResourceProperty(resource, TASK_ID_PROPERTY_ID, response.getTaskId(), requestedIds);
         setResourceProperty(resource, TASK_STAGE_ID_PROPERTY_ID, response.getStageId(), requestedIds);

+ 17 - 11
ambari-server/src/main/java/org/apache/ambari/server/stageplanner/RoleGraph.java

@@ -37,6 +37,9 @@ public class RoleGraph {
   private Stage initialStage = null;
   private boolean sameHostOptimization = true;
 
+  public RoleGraph() {
+  }
+  
   public RoleGraph(RoleCommandOrder rd) {
     this.roleDependencies = rd;
   }
@@ -66,17 +69,19 @@ public class RoleGraph {
       }
     }
 
-    //Add edges
-    for (String roleI : graph.keySet()) {
-      for (String roleJ : graph.keySet()) {
-        if (!roleI.equals(roleJ)) {
-          RoleGraphNode rgnI = graph.get(roleI);
-          RoleGraphNode rgnJ = graph.get(roleJ);
-          int order = roleDependencies.order(rgnI, rgnJ);
-          if (order == -1) {
-            rgnI.addEdge(rgnJ);
-          } else if (order == 1) {
-            rgnJ.addEdge(rgnI);
+    if (null != roleDependencies) {
+      //Add edges
+      for (String roleI : graph.keySet()) {
+        for (String roleJ : graph.keySet()) {
+          if (!roleI.equals(roleJ)) {
+            RoleGraphNode rgnI = graph.get(roleI);
+            RoleGraphNode rgnJ = graph.get(roleJ);
+            int order = roleDependencies.order(rgnI, rgnJ);
+            if (order == -1) {
+              rgnI.addEdge(rgnJ);
+            } else if (order == 1) {
+              rgnJ.addEdge(rgnI);
+            }
           }
         }
       }
@@ -133,6 +138,7 @@ public class RoleGraph {
 
     Stage newStage = new Stage(origStage.getRequestId(),
         origStage.getLogDir(), origStage.getClusterName(),
+        origStage.getClusterId(),
         origStage.getRequestContext(), origStage.getClusterHostInfo());
     newStage.setSuccessFactors(origStage.getSuccessFactors());
     for (RoleGraphNode rgn : stageGraphNodes) {

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

@@ -161,7 +161,7 @@ public class StageUtils {
   //For testing only
   public static Stage getATestStage(long requestId, long stageId, String hostname, String clusterHostInfo) {
 
-    Stage s = new Stage(requestId, "/tmp", "cluster1", "context", clusterHostInfo);
+    Stage s = new Stage(requestId, "/tmp", "cluster1", 1L, "context", clusterHostInfo);
     s.setStageId(stageId);
     long now = System.currentTimeMillis();
     s.addHostRoleExecutionCommand(hostname, Role.NAMENODE, RoleCommand.INSTALL,

+ 1 - 1
ambari-server/src/main/resources/custom_action_definitions/system_action_definitions.xml

@@ -42,7 +42,7 @@
   <actionDefinition>
     <actionName>check_host</actionName>
     <actionType>SYSTEM</actionType>
-    <inputs>threshold</inputs>
+    <inputs></inputs>
     <targetService></targetService>
     <targetComponent></targetComponent>
     <defaultTimeout>60</defaultTimeout>

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

@@ -152,7 +152,7 @@ public class ExecutionCommandWrapperTest {
   
   private static void createTask(ActionDBAccessor db, long requestId, long stageId, String hostName, String clusterName) {
     
-    Stage s = new Stage(requestId, "/var/log", clusterName, "execution command wrapper test", "clusterHostInfo");
+    Stage s = new Stage(requestId, "/var/log", clusterName, 1L, "execution command wrapper test", "clusterHostInfo");
     s.setStageId(stageId);
     s.addHostRoleExecutionCommand(hostName, Role.NAMENODE,
         RoleCommand.START,

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

@@ -319,7 +319,7 @@ public class TestActionDBAccessorImpl {
 
   @Test
   public void testAbortRequest() throws AmbariException {
-    Stage s = new Stage(requestId, "/a/b", "cluster1", "action db accessor test", "clusterHostInfo");
+    Stage s = new Stage(requestId, "/a/b", "cluster1", 1L, "action db accessor test", "clusterHostInfo");
     s.setStageId(stageId);
 
     clusters.addHost("host2");
@@ -383,7 +383,7 @@ public class TestActionDBAccessorImpl {
   }
 
   private Stage createStubStage(String hostname, long requestId, long stageId) {
-    Stage s = new Stage(requestId, "/a/b", "cluster1", "action db accessor test", "clusterHostInfo");
+    Stage s = new Stage(requestId, "/a/b", "cluster1", 1L, "action db accessor test", "clusterHostInfo");
     s.setStageId(stageId);
     s.addHostRoleExecutionCommand(hostname, Role.HBASE_MASTER,
         RoleCommand.START,
@@ -400,7 +400,7 @@ public class TestActionDBAccessorImpl {
 
   private void populateActionDBWithCustomAction(ActionDBAccessor db, String hostname,
                                 long requestId, long stageId) {
-    Stage s = new Stage(requestId, "/a/b", "cluster1", "action db accessor test", "");
+    Stage s = new Stage(requestId, "/a/b", "cluster1", 1L, "action db accessor test", "");
     s.setStageId(stageId);
     s.addHostRoleExecutionCommand(hostname, Role.valueOf(actionName),
         RoleCommand.ACTIONEXECUTE,

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

@@ -160,7 +160,7 @@ public class TestActionManager {
   }
 
   private void populateActionDB(ActionDBAccessor db, String hostname) {
-    Stage s = new Stage(requestId, "/a/b", "cluster1", "action manager test", "clusterHostInfo");
+    Stage s = new Stage(requestId, "/a/b", "cluster1", 1L, "action manager test", "clusterHostInfo");
     s.setStageId(stageId);
     s.addHostRoleExecutionCommand(hostname, Role.HBASE_MASTER,
         RoleCommand.START,

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

@@ -102,10 +102,13 @@ public class TestActionScheduler {
     ServiceComponentHost sch = mock(ServiceComponentHost.class);
     UnitOfWork unitOfWork = mock(UnitOfWork.class);
     when(fsm.getCluster(anyString())).thenReturn(oneClusterMock);
+    when(fsm.getClusterById(anyLong())).thenReturn(oneClusterMock);
     when(oneClusterMock.getService(anyString())).thenReturn(serviceObj);
+    when(oneClusterMock.getClusterId()).thenReturn(Long.valueOf(1L));
     when(serviceObj.getServiceComponent(anyString())).thenReturn(scomp);
     when(scomp.getServiceComponentHost(anyString())).thenReturn(sch);
     when(serviceObj.getCluster()).thenReturn(oneClusterMock);
+    
     String hostname = "ahost.ambari.apache.org";
     Host host = mock(Host.class);
     HashMap<String, ServiceComponentHost> hosts =
@@ -336,7 +339,7 @@ public class TestActionScheduler {
     when(serviceObj.getCluster()).thenReturn(oneClusterMock);
 
     final List<Stage> stages = new ArrayList<Stage>();
-    Stage stage = new Stage(1, "/tmp", "cluster1", "stageWith2Tasks",
+    Stage stage = new Stage(1, "/tmp", "cluster1", 1L, "stageWith2Tasks",
       CLUSTER_HOST_INFO);
     addInstallTaskToStage(stage, hostname1, "cluster1", Role.DATANODE,
       RoleCommand.INSTALL, Service.Type.HDFS, 1);
@@ -565,7 +568,7 @@ public class TestActionScheduler {
 
   private static Stage getStageWithServerAction(long requestId, long stageId, String hostName,
                                                 Map<String, String> payload, String requestContext) {
-    Stage stage = new Stage(requestId, "/tmp", "cluster1", requestContext, CLUSTER_HOST_INFO);
+    Stage stage = new Stage(requestId, "/tmp", "cluster1", 1L, requestContext, CLUSTER_HOST_INFO);
     stage.setStageId(stageId);
     long now = System.currentTimeMillis();
     stage.addServerActionCommand(ServerAction.Command.FINALIZE_UPGRADE, Role.AMBARI_SERVER_ACTION,
@@ -898,7 +901,8 @@ public class TestActionScheduler {
     final List<Stage> stages = new ArrayList<Stage>();
 
     long now = System.currentTimeMillis();
-    Stage stage = new Stage(1, "/tmp", "cluster1", "testRequestFailureBasedOnSuccessFactor", CLUSTER_HOST_INFO);
+    Stage stage = new Stage(1, "/tmp", "cluster1", 1L,
+        "testRequestFailureBasedOnSuccessFactor", CLUSTER_HOST_INFO);
     stage.setStageId(1);
 
     addHostRoleExecutionCommand(now, stage, Role.SQOOP, Service.Type.SQOOP,
@@ -1083,7 +1087,7 @@ public class TestActionScheduler {
     final List<Stage> stages = new ArrayList<Stage>();
 
     long now = System.currentTimeMillis();
-    Stage stage = new Stage(1, "/tmp", "cluster1", "testRequestFailureBasedOnSuccessFactor", CLUSTER_HOST_INFO);
+    Stage stage = new Stage(1, "/tmp", "cluster1", 1L, "testRequestFailureBasedOnSuccessFactor", CLUSTER_HOST_INFO);
     stage.setStageId(1);
     stage.addHostRoleExecutionCommand("host1", Role.DATANODE, RoleCommand.UPGRADE,
         new ServiceComponentHostUpgradeEvent(Role.DATANODE.toString(), "host1", now, "HDP-0.2"),
@@ -1219,7 +1223,7 @@ public class TestActionScheduler {
   private Stage getStageWithSingleTask(String hostname, String clusterName, Role role,
                                        RoleCommand roleCommand, Service.Type service, int taskId,
                                        int stageId, int requestId) {
-    Stage stage = new Stage(requestId, "/tmp", clusterName, "getStageWithSingleTask", CLUSTER_HOST_INFO);
+    Stage stage = new Stage(requestId, "/tmp", clusterName, 1L, "getStageWithSingleTask", CLUSTER_HOST_INFO);
     stage.setStageId(stageId);
     stage.addHostRoleExecutionCommand(hostname, role, roleCommand,
         new ServiceComponentHostUpgradeEvent(role.toString(), hostname, System.currentTimeMillis(), "HDP-0.2"),
@@ -1395,7 +1399,7 @@ public class TestActionScheduler {
     when(serviceObj.getCluster()).thenReturn(oneClusterMock);
 
     final List<Stage> stages = new ArrayList<Stage>();
-    Stage stage1 = new Stage(1, "/tmp", "cluster1", "stageWith2Tasks",
+    Stage stage1 = new Stage(1, "/tmp", "cluster1", 1L, "stageWith2Tasks",
             CLUSTER_HOST_INFO);
     addInstallTaskToStage(stage1, hostname1, "cluster1", Role.HBASE_MASTER,
             RoleCommand.INSTALL, Service.Type.HBASE, 1);

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

@@ -50,7 +50,7 @@ public class TestStage {
   @Test
   public void testGetRequestContext() {
 
-    Stage stage = new Stage(1, "/logDir", "c1", "My Context", CLUSTER_HOST_INFO);
+    Stage stage = new Stage(1, "/logDir", "c1", 1L, "My Context", CLUSTER_HOST_INFO);
     assertEquals("My Context", stage.getRequestContext());
     assertEquals(CLUSTER_HOST_INFO, stage.getClusterHostInfo());
   }

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

@@ -613,7 +613,7 @@ public class TestHeartbeatHandler {
   }
 
   private void populateActionDB(ActionDBAccessor db, String DummyHostname1) {
-    Stage s = new Stage(requestId, "/a/b", DummyCluster, "heartbeat handler test", "clusterHostInfo");
+    Stage s = new Stage(requestId, "/a/b", DummyCluster, 1L, "heartbeat handler test", "clusterHostInfo");
     s.setStageId(stageId);
     String filename = null;
     s.addHostRoleExecutionCommand(DummyHostname1, Role.HBASE_MASTER,
@@ -977,7 +977,7 @@ public class TestHeartbeatHandler {
       getServiceComponent(DATANODE).getServiceComponentHost(DummyHostname1);
     serviceComponentHost1.setState(State.INSTALLING);
 
-    Stage s = new Stage(1, "/a/b", "cluster1", "action manager test",
+    Stage s = new Stage(1, "/a/b", "cluster1", 1L, "action manager test",
       "clusterHostInfo");
     s.setStageId(1);
     s.addHostRoleExecutionCommand(DummyHostname1, Role.DATANODE, RoleCommand.INSTALL,
@@ -1493,7 +1493,7 @@ public class TestHeartbeatHandler {
     serviceComponentHost1.setDesiredStackVersion(stack130);
     serviceComponentHost2.setStackVersion(stack122);
 
-    Stage s = new Stage(requestId, "/a/b", "cluster1", "action manager test",
+    Stage s = new Stage(requestId, "/a/b", "cluster1", 1L, "action manager test",
       "clusterHostInfo");
     s.setStageId(stageId);
     s.addHostRoleExecutionCommand(DummyHostname1, Role.DATANODE, RoleCommand.UPGRADE,

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

@@ -83,6 +83,7 @@ import org.apache.ambari.server.orm.dao.ExecutionCommandDAO;
 import org.apache.ambari.server.orm.dao.HostDAO;
 import org.apache.ambari.server.orm.dao.RoleDAO;
 import org.apache.ambari.server.orm.entities.ExecutionCommandEntity;
+import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.RoleEntity;
 import org.apache.ambari.server.security.authorization.Users;
 import org.apache.ambari.server.serveraction.ServerAction;
@@ -219,12 +220,17 @@ public class AmbariManagementControllerTest {
     host.setHostAttributes(hostAttributes);
   }
   
+  private void addHost(String hostname) throws AmbariException {
+    addHost(hostname, null);
+  }
+  
   private void addHost(String hostname, String clusterName) throws AmbariException {
     clusters.addHost(hostname);
     setOsFamily(clusters.getHost(hostname), "redhat", "6.3");
     clusters.getHost(hostname).setState(HostState.HEALTHY);
     clusters.getHost(hostname).persist();
-    clusters.mapHostToCluster(hostname, clusterName);
+    if (null != clusterName)
+      clusters.mapHostToCluster(hostname, clusterName);
   }
 
   private void createCluster(String clusterName) throws AmbariException {
@@ -7716,7 +7722,7 @@ public class AmbariManagementControllerTest {
 
 
     List<Stage> stages = new ArrayList<Stage>();
-    stages.add(new Stage(requestId1, "/a1", clusterName, context, CLUSTER_HOST_INFO));
+    stages.add(new Stage(requestId1, "/a1", clusterName, 1L, context, CLUSTER_HOST_INFO));
     stages.get(0).setStageId(1);
     stages.get(0).addHostRoleExecutionCommand(hostName1, Role.HBASE_MASTER,
             RoleCommand.START,
@@ -7724,14 +7730,14 @@ public class AmbariManagementControllerTest {
                     hostName1, System.currentTimeMillis()),
             clusterName, "HBASE");
 
-    stages.add(new Stage(requestId1, "/a2", clusterName, context, CLUSTER_HOST_INFO));
+    stages.add(new Stage(requestId1, "/a2", clusterName, 1L, context, CLUSTER_HOST_INFO));
     stages.get(1).setStageId(2);
     stages.get(1).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
             new ServiceComponentHostStartEvent(Role.HBASE_CLIENT.toString(),
                     hostName1, System.currentTimeMillis()), clusterName, "HBASE");
 
-    stages.add(new Stage(requestId1, "/a3", clusterName, context, CLUSTER_HOST_INFO));
+    stages.add(new Stage(requestId1, "/a3", clusterName, 1L, context, CLUSTER_HOST_INFO));
     stages.get(2).setStageId(3);
     stages.get(2).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
@@ -7742,14 +7748,14 @@ public class AmbariManagementControllerTest {
     actionDB.persistActions(request);
 
     stages.clear();
-    stages.add(new Stage(requestId2, "/a4", clusterName, context, CLUSTER_HOST_INFO));
+    stages.add(new Stage(requestId2, "/a4", clusterName, 1L, context, CLUSTER_HOST_INFO));
     stages.get(0).setStageId(4);
     stages.get(0).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
             new ServiceComponentHostStartEvent(Role.HBASE_CLIENT.toString(),
                     hostName1, System.currentTimeMillis()), clusterName, "HBASE");
 
-    stages.add(new Stage(requestId2, "/a5", clusterName, context, CLUSTER_HOST_INFO));
+    stages.add(new Stage(requestId2, "/a5", clusterName, 1L, context, CLUSTER_HOST_INFO));
     stages.get(1).setStageId(5);
     stages.get(1).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
@@ -9972,6 +9978,92 @@ public class AmbariManagementControllerTest {
       // expected
     }
   }
+
+  @Test
+  public void testCreateCustomActionNoCluster() throws Exception {
+    String hostname1 = "h1";
+    String hostname2 = "h2";
+    addHost(hostname1);
+    addHost(hostname2);
+    
+    ambariMetaInfo.addActionDefinition(new ActionDefinition("a1", ActionType.SYSTEM,
+        "", "", "", "action def description", TargetHostType.ANY,
+        Short.valueOf("60")));
+    
+    Map<String, String> requestProperties = new HashMap<String, String>();
+    requestProperties.put(REQUEST_CONTEXT_PROPERTY, "Called from a test");
+    
+    Map<String, String> requestParams = new HashMap<String, String>();
+    requestParams.put("some_custom_param", "abc");
+
+    // !!! target single host
+    List<String> hosts = Arrays.asList(hostname1);
+    RequestResourceFilter resourceFilter = new RequestResourceFilter(null, null, hosts);
+    List<RequestResourceFilter> resourceFilters = new ArrayList<RequestResourceFilter>();
+    resourceFilters.add(resourceFilter);
+    
+    ExecuteActionRequest actionRequest = new ExecuteActionRequest(null, null,
+        "a1", resourceFilters, null, requestParams);
+    RequestStatusResponse response = controller.createAction(actionRequest, requestProperties);
+    assertEquals(1, response.getTasks().size());
+    ShortTaskStatus taskStatus = response.getTasks().get(0);
+    Assert.assertEquals(hostname1, taskStatus.getHostName());
+
+    Stage stage = actionDB.getAllStages(response.getRequestId()).get(0);
+    Assert.assertNotNull(stage);
+    Assert.assertEquals(-1L, stage.getClusterId());
+    
+    List<HostRoleCommand> storedTasks = actionDB.getRequestTasks(response.getRequestId());    
+    Assert.assertEquals(1, storedTasks.size());
+    HostRoleCommand task = storedTasks.get(0);
+    Assert.assertEquals(RoleCommand.ACTIONEXECUTE, task.getRoleCommand());
+    Assert.assertEquals("a1", task.getRole().name());
+    Assert.assertEquals(hostname1, task.getHostName());
+    
+    ExecutionCommand cmd = task.getExecutionCommandWrapper().getExecutionCommand();
+    Assert.assertTrue(cmd.getCommandParams().containsKey("some_custom_param"));
+    Assert.assertEquals(null, cmd.getServiceName());
+    Assert.assertEquals(null, cmd.getComponentName());
+    
+    // !!! target two hosts
+
+    hosts = Arrays.asList(hostname1, hostname2);
+    resourceFilter = new RequestResourceFilter(null, null, hosts);
+    resourceFilters = new ArrayList<RequestResourceFilter>();
+    resourceFilters.add(resourceFilter);
+    
+    actionRequest = new ExecuteActionRequest(null, null,
+        "a1", resourceFilters, null, requestParams);
+    response = controller.createAction(actionRequest, requestProperties);
+    assertEquals(2, response.getTasks().size());
+    boolean host1Found = false;
+    boolean host2Found = false;
+    for (ShortTaskStatus sts : response.getTasks()) {
+      if (sts.getHostName().equals(hostname1))
+        host1Found = true;
+      else if (sts.getHostName().equals(hostname2))
+        host2Found = true;
+    }
+    Assert.assertTrue(host1Found);
+    Assert.assertTrue(host2Found);
+
+    stage = actionDB.getAllStages(response.getRequestId()).get(0);
+    Assert.assertNotNull(stage);
+    Assert.assertEquals(-1L, stage.getClusterId());
+    
+    storedTasks = actionDB.getRequestTasks(response.getRequestId());    
+    Assert.assertEquals(2, storedTasks.size());
+    task = storedTasks.get(0);
+    Assert.assertEquals(RoleCommand.ACTIONEXECUTE, task.getRoleCommand());
+    Assert.assertEquals("a1", task.getRole().name());
+    Assert.assertEquals(hostname1, task.getHostName());
+    
+    cmd = task.getExecutionCommandWrapper().getExecutionCommand();
+    Assert.assertTrue(cmd.getCommandParams().containsKey("some_custom_param"));
+    Assert.assertEquals(null, cmd.getServiceName());
+    Assert.assertEquals(null, cmd.getComponentName());
+  }
+
 }
 
   

+ 137 - 21
ambari-server/src/test/java/org/apache/ambari/server/controller/internal/RequestResourceProviderTest.java

@@ -18,6 +18,27 @@
 
 package org.apache.ambari.server.controller.internal;
 
+import static org.easymock.EasyMock.capture;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.createNiceMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.reset;
+import static org.easymock.EasyMock.verify;
+import static org.easymock.EasyMock.anyObject;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.actionmanager.HostRoleCommand;
@@ -30,10 +51,7 @@ import org.apache.ambari.server.controller.spi.NoSuchParentResourceException;
 import org.apache.ambari.server.controller.spi.Predicate;
 import org.apache.ambari.server.controller.spi.Request;
 import org.apache.ambari.server.controller.spi.Resource;
-import org.apache.ambari.server.controller.spi.ResourceAlreadyExistsException;
 import org.apache.ambari.server.controller.spi.ResourceProvider;
-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.state.Cluster;
@@ -41,24 +59,6 @@ import org.apache.ambari.server.state.Clusters;
 import org.easymock.Capture;
 import org.junit.Assert;
 import org.junit.Test;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import static org.easymock.EasyMock.capture;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.createNiceMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.reset;
-import static org.easymock.EasyMock.verify;
 
 /**
  * RequestResourceProvider tests.
@@ -1062,5 +1062,121 @@ public class RequestResourceProviderTest {
     Assert.assertEquals(level.getHostName(), host_id);
   }
 
+  @Test
+  public void testCreateResourcesForNonCluster() throws Exception {
+    Resource.Type type = Resource.Type.Request;
+
+    Capture<ExecuteActionRequest> actionRequest = new Capture<ExecuteActionRequest>();
+    Capture<HashMap<String, String>> propertyMap = new Capture<HashMap<String, String>>();
+
+    AmbariManagementController managementController = createMock(AmbariManagementController.class);
+    RequestStatusResponse response = createNiceMock(RequestStatusResponse.class);
+
+    expect(managementController.createAction(capture(actionRequest), capture(propertyMap)))
+        .andReturn(response).anyTimes();
+
+    // replay
+    replay(managementController);
+
+    // add the property map to a set for the request.  add more maps for multiple creates
+    Set<Map<String, Object>> propertySet = new LinkedHashSet<Map<String, Object>>();
+
+    Map<String, Object> properties = new LinkedHashMap<String, Object>();
+
+    Set<Map<String, Object>> filterSet = new HashSet<Map<String, Object>>();
+    Map<String, Object> filterMap = new HashMap<String, Object>();
+    filterMap.put(RequestResourceProvider.HOSTS_ID, "h1,h2");
+    filterSet.add(filterMap);
+
+    properties.put(RequestResourceProvider.REQUEST_RESOURCE_FILTER_ID, filterSet);
+
+    propertySet.add(properties);
+
+    Map<String, String> requestInfoProperties = new HashMap<String, String>();
+    requestInfoProperties.put(RequestResourceProvider.ACTION_ID, "check_java");
+
+    // create the request
+    Request request = PropertyHelper.getCreateRequest(propertySet, requestInfoProperties);
+    ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
+        type,
+        PropertyHelper.getPropertyIds(type),
+        PropertyHelper.getKeyPropertyIds(type),
+        managementController);
+    provider.createResources(request);
+    ExecuteActionRequest capturedRequest = actionRequest.getValue();
+
+    Assert.assertTrue(actionRequest.hasCaptured());
+    Assert.assertFalse("expected an action", Boolean.valueOf(capturedRequest.isCommand()));
+    Assert.assertEquals("check_java", capturedRequest.getActionName());
+    Assert.assertEquals(null, capturedRequest.getCommandName());
+    Assert.assertNotNull(capturedRequest.getResourceFilters());
+    Assert.assertEquals(1, capturedRequest.getResourceFilters().size());
+    RequestResourceFilter capturedResourceFilter = capturedRequest.getResourceFilters().get(0);
+    Assert.assertEquals(null, capturedResourceFilter.getServiceName());
+    Assert.assertEquals(null, capturedResourceFilter.getComponentName());
+    Assert.assertNotNull(capturedResourceFilter.getHostNames());
+    Assert.assertEquals(2, capturedResourceFilter.getHostNames().size());
+    Assert.assertEquals(0, actionRequest.getValue().getParameters().size());
+    }
+  
+  @Test
+  public void testGetResourcesWithoutCluster() throws Exception {
+    Resource.Type type = Resource.Type.Request;
+
+    AmbariManagementController managementController = createMock(AmbariManagementController.class);
+    ActionManager actionManager = createNiceMock(ActionManager.class);
+    HostRoleCommand hostRoleCommand = createNiceMock(HostRoleCommand.class);
+    Clusters clusters = createNiceMock(Clusters.class);
+
+    List<HostRoleCommand> hostRoleCommands = new LinkedList<HostRoleCommand>();
+    hostRoleCommands.add(hostRoleCommand);
+
+    org.apache.ambari.server.actionmanager.Request requestMock =
+        createNiceMock(org.apache.ambari.server.actionmanager.Request.class);
+    expect(requestMock.getCommands()).andReturn(hostRoleCommands).anyTimes();
+    expect(requestMock.getRequestContext()).andReturn("this is a context").anyTimes();
+    expect(requestMock.getClusterName()).andReturn(null).anyTimes();
+    expect(requestMock.getRequestId()).andReturn(100L).anyTimes();
+
+    Capture<Collection<Long>> requestIdsCapture = new Capture<Collection<Long>>();
+
+    // set expectations
+    expect(managementController.getActionManager()).andReturn(actionManager).anyTimes();
+    expect(managementController.getClusters()).andReturn(clusters).anyTimes();
+    expect(clusters.getCluster(anyObject(String.class))).andReturn(null).anyTimes();
+    expect(actionManager.getRequests(capture(requestIdsCapture))).andReturn(Collections.singletonList(requestMock));
+    expect(hostRoleCommand.getRequestId()).andReturn(100L).anyTimes();
+    expect(hostRoleCommand.getStatus()).andReturn(HostRoleStatus.IN_PROGRESS);
+
+    // replay
+    replay(managementController, actionManager, hostRoleCommand, clusters, requestMock);
+
+    ResourceProvider provider = AbstractControllerResourceProvider.getResourceProvider(
+        type,
+        PropertyHelper.getPropertyIds(type),
+        PropertyHelper.getKeyPropertyIds(type),
+        managementController);
+
+    Set<String> propertyIds = new HashSet<String>();
+
+    propertyIds.add(RequestResourceProvider.REQUEST_ID_PROPERTY_ID);
+    propertyIds.add(RequestResourceProvider.REQUEST_STATUS_PROPERTY_ID);
+
+    Predicate predicate = new PredicateBuilder().
+        property(RequestResourceProvider.REQUEST_ID_PROPERTY_ID).equals("100").
+        toPredicate();
+    Request request = PropertyHelper.getReadRequest(propertyIds);
+    Set<Resource> resources = provider.getResources(request, predicate);
+
+    Assert.assertEquals(1, resources.size());
+    for (Resource resource : resources) {
+      Assert.assertEquals(100L, (long) (Long) resource.getPropertyValue(RequestResourceProvider.REQUEST_ID_PROPERTY_ID));
+      Assert.assertEquals("IN_PROGRESS", resource.getPropertyValue(RequestResourceProvider.REQUEST_STATUS_PROPERTY_ID));
+      Assert.assertNull(resource.getPropertyValue(RequestResourceProvider.REQUEST_CLUSTER_NAME_PROPERTY_ID));
+    }
+
+    // verify
+    verify(managementController, actionManager, hostRoleCommand, clusters);    
+  }
 
 }