Przeglądaj źródła

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

Nate Cole 11 lat temu
rodzic
commit
4c04bf8720
24 zmienionych plików z 540 dodań i 239 usunięć
  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:
     for command in commands:
       if not command.has_key('serviceName'):
       if not command.has_key('serviceName'):
         command['serviceName'] = "null"
         command['serviceName'] = "null"
+      if not command.has_key('clusterName'):
+        command['clusterName'] = 'null'
+
       logger.info("Adding " + command['commandType'] + " for service " + \
       logger.info("Adding " + command['commandType'] + " for service " + \
                   command['serviceName'] + " of cluster " + \
                   command['serviceName'] + " of cluster " + \
                   command['clusterName'] + " to the queue.")
                   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 ||
       if (command.getStatus() == HostRoleStatus.QUEUED ||
           command.getStatus() == HostRoleStatus.IN_PROGRESS ||
           command.getStatus() == HostRoleStatus.IN_PROGRESS ||
           command.getStatus() == HostRoleStatus.PENDING) {
           command.getStatus() == HostRoleStatus.PENDING) {
+
         command.setStatus(HostRoleStatus.ABORTED);
         command.setStatus(HostRoleStatus.ABORTED);
         command.setEndTime(now);
         command.setEndTime(now);
         LOG.info("Aborting command. Hostname " + command.getHostName()
         LOG.info("Aborting command. Hostname " + command.getHostName()
@@ -203,11 +204,13 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
 
 
     RequestEntity requestEntity = request.constructNewPersistenceEntity();
     RequestEntity requestEntity = request.constructNewPersistenceEntity();
 
 
+    Long clusterId = Long.valueOf(-1L);
     ClusterEntity clusterEntity = clusterDAO.findById(request.getClusterId());
     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);
     requestDAO.create(requestEntity);
 
 
     //TODO wire request to cluster
     //TODO wire request to cluster
@@ -216,7 +219,7 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
     for (Stage stage : request.getStages()) {
     for (Stage stage : request.getStages()) {
       StageEntity stageEntity = stage.constructNewPersistenceEntity();
       StageEntity stageEntity = stage.constructNewPersistenceEntity();
       stageEntities.add(stageEntity);
       stageEntities.add(stageEntity);
-      stageEntity.setClusterId(clusterEntity.getClusterId());
+      stageEntity.setClusterId(clusterId);
       //TODO refactor to reduce merges
       //TODO refactor to reduce merges
       stageEntity.setRequest(requestEntity);
       stageEntity.setRequest(requestEntity);
       stageDAO.create(stageEntity);
       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) {
     if (taskTimeoutAdjustment) {
       taskTimeout = actionTimeout + s.getStageTimeout();
       taskTimeout = actionTimeout + s.getStageTimeout();
     }
     }
+
+    Cluster cluster = null;
+    if (null != s.getClusterName()) {
+      cluster = fsmObject.getCluster(s.getClusterName());
+    }
+
     for (String host : s.getHosts()) {
     for (String host : s.getHosts()) {
       List<ExecutionCommandWrapper> commandWrappers = s.getExecutionCommands(host);
       List<ExecutionCommandWrapper> commandWrappers = s.getExecutionCommands(host);
-      Cluster cluster = fsmObject.getCluster(s.getClusterName());
       Host hostObj = fsmObject.getHost(host);
       Host hostObj = fsmObject.getHost(host);
+
       for(ExecutionCommandWrapper wrapper : commandWrappers) {
       for(ExecutionCommandWrapper wrapper : commandWrappers) {
         ExecutionCommand c = wrapper.getExecutionCommand();
         ExecutionCommand c = wrapper.getExecutionCommand();
         String roleStr = c.getRole();
         String roleStr = c.getRole();
         HostRoleStatus status = s.getHostRoleStatus(host, roleStr);
         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
         // Check that service host component is not deleted
-        if (scHosts!= null && ! scHosts.containsKey(host)) {
+        if (hostDeleted) {
+          
           String message = String.format(
           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
           // 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;
           status = HostRoleStatus.ABORTED;
-        } else if (timeOutActionNeeded(status, s, hostObj, roleStr, now,
-                taskTimeout)) {
+        } else if (timeOutActionNeeded(status, s, hostObj, roleStr, now, taskTimeout)) {
           // Process command timeouts
           // 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) {
           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());
             db.timeoutHostRole(host, s.getRequestId(), s.getStageId(), c.getRole());
             //Reinitialize status
             //Reinitialize status
             status = s.getHostRoleStatus(host, roleStr);
             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
             // Dequeue command
             actionQueue.dequeue(host, c.getCommandId());
             actionQueue.dequeue(host, c.getCommandId());
@@ -445,6 +460,7 @@ class ActionScheduler implements Runnable {
           //Need to schedule first time
           //Need to schedule first time
           commandsToSchedule.add(c);
           commandsToSchedule.add(c);
         }
         }
+
         this.updateRoleStats(status, roleStats.get(roleStr));
         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) {
   public Request(@Assisted long requestId, @Assisted("clusterId") Long clusterId, Clusters clusters) {
     this.requestId = requestId;
     this.requestId = requestId;
-    this.clusterId = clusterId;
+    this.clusterId = clusterId.longValue();
     this.createTime = System.currentTimeMillis();
     this.createTime = System.currentTimeMillis();
     this.startTime = -1;
     this.startTime = -1;
     this.endTime = -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();
       this.clusterName = stage.getClusterName();
       try {
       try {
         this.clusterId = clusters.getCluster(clusterName).getClusterId();
         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.requestContext = stages.iterator().next().getRequestContext();
       this.createTime = System.currentTimeMillis();
       this.createTime = System.currentTimeMillis();
@@ -261,7 +266,7 @@ public class Request {
 
 
 
 
   public Long getClusterId() {
   public Long getClusterId() {
-    return clusterId;
+    return Long.valueOf(clusterId);
   }
   }
 
 
   public String getClusterName() {
   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 static Logger LOG = LoggerFactory.getLogger(Stage.class);
   private final long requestId;
   private final long requestId;
   private String clusterName;
   private String clusterName;
+  private long clusterId = -1L;
   private long stageId = -1;
   private long stageId = -1;
   private final String logDir;
   private final String logDir;
   private final String requestContext;
   private final String requestContext;
@@ -73,12 +74,17 @@ public class Stage {
       new TreeMap<String, List<ExecutionCommandWrapper>>();
       new TreeMap<String, List<ExecutionCommandWrapper>>();
 
 
   @AssistedInject
   @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.wrappersLoaded = true;
     this.requestId = requestId;
     this.requestId = requestId;
     this.logDir = logDir;
     this.logDir = logDir;
     this.clusterName = clusterName;
     this.clusterName = clusterName;
+    this.clusterId = clusterId;
     this.requestContext = requestContext == null ? "" : requestContext;
     this.requestContext = requestContext == null ? "" : requestContext;
     this.clusterHostInfo = clusterHostInfo;
     this.clusterHostInfo = clusterHostInfo;
   }
   }
@@ -349,6 +355,11 @@ public class Stage {
   public String getClusterName() {
   public String getClusterName() {
     return clusterName;
     return clusterName;
   }
   }
+  
+  public long getClusterId() {
+    return clusterId;
+  }
+  
 
 
   public String getRequestContext() {
   public String getRequestContext() {
     return requestContext;
     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;
 package org.apache.ambari.server.actionmanager;
 
 
-import com.google.inject.assistedinject.Assisted;
 import org.apache.ambari.server.orm.entities.StageEntity;
 import org.apache.ambari.server.orm.entities.StageEntity;
 
 
+import com.google.inject.assistedinject.Assisted;
+
 public interface StageFactory {
 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);
   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) {
     for (Resource.Type type : types) {
       String resourceKeyProp = schema.getKeyPropertyId(type);
       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;
     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 {
   private class RequestHrefPostProcessor implements PostProcessor {
     @Override
     @Override
     public void process(Request request, TreeNode<Resource> resultNode, String href) {
     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().
       Object requestId = resultNode.getObject().getPropertyValue(getClusterController().
           getSchema(Resource.Type.Request).getKeyPropertyId(Resource.Type.Request));
           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());
       resultNode.setProperty("href", sb.toString());
     }
     }
   }
   }
 
 
   private class RequestSourceScheduleHrefPostProcessor implements PostProcessor {
   private class RequestSourceScheduleHrefPostProcessor implements PostProcessor {
 
 
-    @SuppressWarnings("unchecked")
     @Override
     @Override
     public void process(Request request, TreeNode<Resource> resultNode, String href) {
     public void process(Request request, TreeNode<Resource> resultNode, String href) {
       StringBuilder sb = new StringBuilder();
       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.
  * Service responsible for request resource requests.
  */
  */
+@Path("/requests/")
 public class RequestService extends BaseService {
 public class RequestService extends BaseService {
   /**
   /**
    * Parent cluster name.
    * Parent cluster name.
@@ -45,6 +46,9 @@ public class RequestService extends BaseService {
   private String m_clusterName;
   private String m_clusterName;
 
 
 
 
+  public RequestService() {
+  }
+  
   /**
   /**
    * Constructor.
    * 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.
    * Get a specific request.
    *
    *
    * @param headers    http headers
    * @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.
    * Get all requests for a cluster.
    *
    *
    * @param headers http headers
    * @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.
    * Create multiple services.
    *
    *
    * @param body        http body
    * @param body        http body
@@ -124,7 +129,9 @@ public class RequestService extends BaseService {
    */
    */
   ResourceInstance createRequestResource(String clusterName, String requestId) {
   ResourceInstance createRequestResource(String clusterName, String requestId) {
     Map<Resource.Type,String> mapIds = new HashMap<Resource.Type, String>();
     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);
     mapIds.put(Resource.Type.Request, requestId);
 
 
     return createResource(Resource.Type.Request, mapIds);
     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;
 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.AmbariException;
 import org.apache.ambari.server.Role;
 import org.apache.ambari.server.Role;
 import org.apache.ambari.server.RoleCommand;
 import org.apache.ambari.server.RoleCommand;
@@ -43,16 +52,8 @@ import org.apache.ambari.server.utils.StageUtils;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 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
  * Helper class containing logic to process custom action execution requests
@@ -63,15 +64,11 @@ public class AmbariActionExecutionHelper {
       LoggerFactory.getLogger(AmbariActionExecutionHelper.class);
       LoggerFactory.getLogger(AmbariActionExecutionHelper.class);
   private static final String TYPE_PYTHON = "PYTHON";
   private static final String TYPE_PYTHON = "PYTHON";
 
 
-  @Inject
-  private ActionMetadata actionMetadata;
   @Inject
   @Inject
   private Clusters clusters;
   private Clusters clusters;
   @Inject
   @Inject
   private AmbariManagementController managementController;
   private AmbariManagementController managementController;
   @Inject
   @Inject
-  private ActionManager actionManager;
-  @Inject
   private AmbariMetaInfo ambariMetaInfo;
   private AmbariMetaInfo ambariMetaInfo;
   @Inject
   @Inject
   private MaintenanceStateHelper maintenanceStateHelper;
   private MaintenanceStateHelper maintenanceStateHelper;
@@ -82,13 +79,9 @@ public class AmbariActionExecutionHelper {
    * @throws AmbariException
    * @throws AmbariException
    */
    */
   public void validateAction(ExecuteActionRequest actionRequest) 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()) {
     if (actionRequest.getActionName() == null || actionRequest.getActionName().isEmpty()) {
       throw new AmbariException("Action name must be specified");
       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();
     List<RequestResourceFilter> resourceFilters = actionRequest.getResourceFilters();
-    String targetService = "";
-    String targetComponent = "";
     RequestResourceFilter resourceFilter = null;
     RequestResourceFilter resourceFilter = null;
-
     if (resourceFilters != null && !resourceFilters.isEmpty()) {
     if (resourceFilters != null && !resourceFilters.isEmpty()) {
       if (resourceFilters.size() > 1) {
       if (resourceFilters.size() > 1) {
         throw new AmbariException("Custom action definition only allows one " +
         throw new AmbariException("Custom action definition only allows one " +
           "resource filter to be specified.");
           "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)) {
       if (!expectedService.isEmpty() && !actualService.isEmpty() && !expectedService.equals(actualService)) {
         throw new AmbariException("Action " + actionRequest.getActionName() + " targets service " + actualService +
         throw new AmbariException("Action " + actionRequest.getActionName() + " targets service " + actualService +
           " that does not match with expected " + expectedService);
           " that does not match with expected " + expectedService);
       }
       }
-
+  
       targetService = expectedService;
       targetService = expectedService;
       if (targetService == null || targetService.isEmpty()) {
       if (targetService == null || targetService.isEmpty()) {
         targetService = actualService;
         targetService = actualService;
       }
       }
-
+  
       if (targetService != null && !targetService.isEmpty()) {
       if (targetService != null && !targetService.isEmpty()) {
         ServiceInfo serviceInfo;
         ServiceInfo serviceInfo;
         try {
         try {
@@ -156,14 +159,14 @@ public class AmbariActionExecutionHelper {
             " targets service " + targetService + " that does not exist.");
             " targets service " + targetService + " that does not exist.");
         }
         }
       }
       }
-
+  
       String expectedComponent = actionDef.getTargetComponent() == null ? "" : actionDef.getTargetComponent();
       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)) {
       if (!expectedComponent.isEmpty() && !actualComponent.isEmpty() && !expectedComponent.equals(actualComponent)) {
         throw new AmbariException("Action " + actionRequest.getActionName() + " targets component " + actualComponent +
         throw new AmbariException("Action " + actionRequest.getActionName() + " targets component " + actualComponent +
           " that does not match with expected " + expectedComponent);
           " that does not match with expected " + expectedComponent);
       }
       }
-
+  
       targetComponent = expectedComponent;
       targetComponent = expectedComponent;
       if (targetComponent == null || targetComponent.isEmpty()) {
       if (targetComponent == null || targetComponent.isEmpty()) {
         targetComponent = actualComponent;
         targetComponent = actualComponent;
@@ -173,7 +176,7 @@ public class AmbariActionExecutionHelper {
         throw new AmbariException("Action " + actionRequest.getActionName() + " targets component " + targetComponent +
         throw new AmbariException("Action " + actionRequest.getActionName() + " targets component " + targetComponent +
           " without specifying the target service.");
           " without specifying the target service.");
       }
       }
-
+  
       if (targetComponent != null && !targetComponent.isEmpty()) {
       if (targetComponent != null && !targetComponent.isEmpty()) {
         ComponentInfo compInfo;
         ComponentInfo compInfo;
         try {
         try {
@@ -189,7 +192,7 @@ public class AmbariActionExecutionHelper {
         }
         }
       }
       }
     }
     }
-
+        
     if (TargetHostType.SPECIFIC.equals(actionDef.getTargetType())
     if (TargetHostType.SPECIFIC.equals(actionDef.getTargetType())
       || (targetService.isEmpty() && targetComponent.isEmpty())) {
       || (targetService.isEmpty() && targetComponent.isEmpty())) {
       if (resourceFilter == null || resourceFilter.getHostNames().size() == 0) {
       if (resourceFilter == null || resourceFilter.getHostNames().size() == 0) {
@@ -213,7 +216,11 @@ public class AmbariActionExecutionHelper {
 
 
     String actionName = actionContext.getActionName();
     String actionName = actionContext.getActionName();
     String clusterName = actionContext.getClusterName();
     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();
     List<RequestResourceFilter> resourceFilters = actionContext.getResourceFilters();
 
 
@@ -227,36 +234,38 @@ public class AmbariActionExecutionHelper {
 
 
     String serviceName = actionContext.getExpectedServiceName();
     String serviceName = actionContext.getExpectedServiceName();
     String componentName = actionContext.getExpectedComponentName();
     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 =
           Map<String, ServiceComponentHost> componentHosts =
             cluster.getService(serviceName)
             cluster.getService(serviceName)
-              .getServiceComponent(component).getServiceComponentHosts();
+              .getServiceComponent(componentName).getServiceComponentHosts();
           candidateHosts.addAll(componentHosts.keySet());
           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 request did not specify hosts and there exists no host
     if (resourceFilter.getHostNames().isEmpty() && candidateHosts.isEmpty()) {
     if (resourceFilter.getHostNames().isEmpty() && candidateHosts.isEmpty()) {
       throw new AmbariException("Suitable hosts not found, component="
       throw new AmbariException("Suitable hosts not found, component="
         + componentName + ", service=" + serviceName
         + 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
     // 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
     //create tasks for each host
     for (String hostName : targetHosts) {
     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>> configurations = new TreeMap<String, Map<String, String>>();
       Map<String, Map<String, String>> configTags = null;
       Map<String, Map<String, String>> configTags = null;
-      if (!serviceName.isEmpty()) {
+      if (!serviceName.isEmpty() && null != cluster) {
         configTags = managementController.findConfigurationTagsWithOverrides(cluster, hostName);
         configTags = managementController.findConfigurationTagsWithOverrides(cluster, hostName);
       }
       }
 
 
@@ -343,8 +354,9 @@ public class AmbariActionExecutionHelper {
       execCmd.setRoleParams(roleParams);
       execCmd.setRoleParams(roleParams);
 
 
       // Generate cluster host info
       // 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
       // cluster passive map
       execCmd.setPassiveInfo(maintenanceSCHs);
       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) {
   private Stage createNewStage(long id, Cluster cluster, long requestId, String requestContext, String clusterHostInfo) {
     String logDir = BASE_LOG_DIR + File.pathSeparator + requestId;
     String logDir = BASE_LOG_DIR + File.pathSeparator + requestId;
     Stage stage =
     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);
     stage.setStageId(id);
     return stage;
     return stage;
   }
   }
@@ -2541,7 +2544,8 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
   public RequestStatusResponse createAction(ExecuteActionRequest actionRequest,
   public RequestStatusResponse createAction(ExecuteActionRequest actionRequest,
       Map<String, String> requestProperties)
       Map<String, String> requestProperties)
       throws AmbariException {
       throws AmbariException {
-    String clusterName;
+    String clusterName = actionRequest.getClusterName();
+        
     String requestContext = "";
     String requestContext = "";
 
 
     if (requestProperties != null) {
     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);
     ActionExecutionContext actionExecContext = getActionExecutionContext(actionRequest);
     if (actionRequest.isCommand()) {
     if (actionRequest.isCommand()) {
       customCommandExecutionHelper.validateAction(actionRequest);
       customCommandExecutionHelper.validateAction(actionRequest);
     } else {
     } else {
       actionExecutionHelper.validateAction(actionRequest);
       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);
         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);
     Stage stage = createNewStage(0, cluster, actionManager.getNextRequestId(), requestContext, clusterHostInfoJson);
 
 
-    Map<String, String> params = createDefaultHostParams(cluster);
-
     if (actionRequest.isCommand()) {
     if (actionRequest.isCommand()) {
       customCommandExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage,
       customCommandExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage,
           params, requestProperties);
           params, requestProperties);
     } else {
     } else {
       actionExecutionHelper.addExecutionCommandsToStage(actionExecContext, stage, params);
       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();
     List<Stage> stages = rg.getStages();
+    
     if (stages != null && !stages.isEmpty()) {
     if (stages != null && !stages.isEmpty()) {
       actionManager.sendActions(stages, actionRequest);
       actionManager.sendActions(stages, actionRequest);
       return getRequestStatusResponse(stage.getRequestId());
       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.SystemException;
 import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
 import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
 import org.apache.ambari.server.state.Clusters;
 import org.apache.ambari.server.state.Clusters;
+
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
@@ -138,22 +139,30 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
     Integer maxResults = (maxResultsRaw == null ? null : Integer.parseInt(maxResultsRaw));
     Integer maxResults = (maxResultsRaw == null ? null : Integer.parseInt(maxResultsRaw));
     Boolean ascOrder = (ascOrderRaw == null ? null : Boolean.parseBoolean(ascOrderRaw));
     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;
     return resources;
   }
   }
 
 
@@ -335,7 +344,9 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
     Map<Long, Resource> resourceMap = new HashMap<Long, Resource>();
     Map<Long, Resource> resourceMap = new HashMap<Long, Resource>();
 
 
     for (org.apache.ambari.server.actionmanager.Request request : requests) {
     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();
     return resourceMap.values();
@@ -345,7 +356,9 @@ public class RequestResourceProvider extends AbstractControllerResourceProvider
                                       Set<String> requestedPropertyIds) {
                                       Set<String> requestedPropertyIds) {
     Resource resource = new ResourceImpl(Resource.Type.Request);
     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_ID_PROPERTY_ID, request.getRequestId(), requestedPropertyIds);
     setResourceProperty(resource, REQUEST_CONTEXT_ID, request.getRequestContext(), requestedPropertyIds);
     setResourceProperty(resource, REQUEST_CONTEXT_ID, request.getRequestContext(), requestedPropertyIds);
     setResourceProperty(resource, REQUEST_TYPE_ID, request.getRequestType(), 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) {
       for (TaskStatusResponse response : responses) {
         Resource resource = new ResourceImpl(Resource.Type.Task);
         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_REQUEST_ID_PROPERTY_ID, response.getRequestId(), requestedIds);
         setResourceProperty(resource, TASK_ID_PROPERTY_ID, response.getTaskId(), requestedIds);
         setResourceProperty(resource, TASK_ID_PROPERTY_ID, response.getTaskId(), requestedIds);
         setResourceProperty(resource, TASK_STAGE_ID_PROPERTY_ID, response.getStageId(), 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 Stage initialStage = null;
   private boolean sameHostOptimization = true;
   private boolean sameHostOptimization = true;
 
 
+  public RoleGraph() {
+  }
+  
   public RoleGraph(RoleCommandOrder rd) {
   public RoleGraph(RoleCommandOrder rd) {
     this.roleDependencies = 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(),
     Stage newStage = new Stage(origStage.getRequestId(),
         origStage.getLogDir(), origStage.getClusterName(),
         origStage.getLogDir(), origStage.getClusterName(),
+        origStage.getClusterId(),
         origStage.getRequestContext(), origStage.getClusterHostInfo());
         origStage.getRequestContext(), origStage.getClusterHostInfo());
     newStage.setSuccessFactors(origStage.getSuccessFactors());
     newStage.setSuccessFactors(origStage.getSuccessFactors());
     for (RoleGraphNode rgn : stageGraphNodes) {
     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
   //For testing only
   public static Stage getATestStage(long requestId, long stageId, String hostname, String clusterHostInfo) {
   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);
     s.setStageId(stageId);
     long now = System.currentTimeMillis();
     long now = System.currentTimeMillis();
     s.addHostRoleExecutionCommand(hostname, Role.NAMENODE, RoleCommand.INSTALL,
     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>
   <actionDefinition>
     <actionName>check_host</actionName>
     <actionName>check_host</actionName>
     <actionType>SYSTEM</actionType>
     <actionType>SYSTEM</actionType>
-    <inputs>threshold</inputs>
+    <inputs></inputs>
     <targetService></targetService>
     <targetService></targetService>
     <targetComponent></targetComponent>
     <targetComponent></targetComponent>
     <defaultTimeout>60</defaultTimeout>
     <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) {
   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.setStageId(stageId);
     s.addHostRoleExecutionCommand(hostName, Role.NAMENODE,
     s.addHostRoleExecutionCommand(hostName, Role.NAMENODE,
         RoleCommand.START,
         RoleCommand.START,

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

@@ -319,7 +319,7 @@ public class TestActionDBAccessorImpl {
 
 
   @Test
   @Test
   public void testAbortRequest() throws AmbariException {
   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);
     s.setStageId(stageId);
 
 
     clusters.addHost("host2");
     clusters.addHost("host2");
@@ -383,7 +383,7 @@ public class TestActionDBAccessorImpl {
   }
   }
 
 
   private Stage createStubStage(String hostname, long requestId, long stageId) {
   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.setStageId(stageId);
     s.addHostRoleExecutionCommand(hostname, Role.HBASE_MASTER,
     s.addHostRoleExecutionCommand(hostname, Role.HBASE_MASTER,
         RoleCommand.START,
         RoleCommand.START,
@@ -400,7 +400,7 @@ public class TestActionDBAccessorImpl {
 
 
   private void populateActionDBWithCustomAction(ActionDBAccessor db, String hostname,
   private void populateActionDBWithCustomAction(ActionDBAccessor db, String hostname,
                                 long requestId, long stageId) {
                                 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.setStageId(stageId);
     s.addHostRoleExecutionCommand(hostname, Role.valueOf(actionName),
     s.addHostRoleExecutionCommand(hostname, Role.valueOf(actionName),
         RoleCommand.ACTIONEXECUTE,
         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) {
   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.setStageId(stageId);
     s.addHostRoleExecutionCommand(hostname, Role.HBASE_MASTER,
     s.addHostRoleExecutionCommand(hostname, Role.HBASE_MASTER,
         RoleCommand.START,
         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);
     ServiceComponentHost sch = mock(ServiceComponentHost.class);
     UnitOfWork unitOfWork = mock(UnitOfWork.class);
     UnitOfWork unitOfWork = mock(UnitOfWork.class);
     when(fsm.getCluster(anyString())).thenReturn(oneClusterMock);
     when(fsm.getCluster(anyString())).thenReturn(oneClusterMock);
+    when(fsm.getClusterById(anyLong())).thenReturn(oneClusterMock);
     when(oneClusterMock.getService(anyString())).thenReturn(serviceObj);
     when(oneClusterMock.getService(anyString())).thenReturn(serviceObj);
+    when(oneClusterMock.getClusterId()).thenReturn(Long.valueOf(1L));
     when(serviceObj.getServiceComponent(anyString())).thenReturn(scomp);
     when(serviceObj.getServiceComponent(anyString())).thenReturn(scomp);
     when(scomp.getServiceComponentHost(anyString())).thenReturn(sch);
     when(scomp.getServiceComponentHost(anyString())).thenReturn(sch);
     when(serviceObj.getCluster()).thenReturn(oneClusterMock);
     when(serviceObj.getCluster()).thenReturn(oneClusterMock);
+    
     String hostname = "ahost.ambari.apache.org";
     String hostname = "ahost.ambari.apache.org";
     Host host = mock(Host.class);
     Host host = mock(Host.class);
     HashMap<String, ServiceComponentHost> hosts =
     HashMap<String, ServiceComponentHost> hosts =
@@ -336,7 +339,7 @@ public class TestActionScheduler {
     when(serviceObj.getCluster()).thenReturn(oneClusterMock);
     when(serviceObj.getCluster()).thenReturn(oneClusterMock);
 
 
     final List<Stage> stages = new ArrayList<Stage>();
     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);
       CLUSTER_HOST_INFO);
     addInstallTaskToStage(stage, hostname1, "cluster1", Role.DATANODE,
     addInstallTaskToStage(stage, hostname1, "cluster1", Role.DATANODE,
       RoleCommand.INSTALL, Service.Type.HDFS, 1);
       RoleCommand.INSTALL, Service.Type.HDFS, 1);
@@ -565,7 +568,7 @@ public class TestActionScheduler {
 
 
   private static Stage getStageWithServerAction(long requestId, long stageId, String hostName,
   private static Stage getStageWithServerAction(long requestId, long stageId, String hostName,
                                                 Map<String, String> payload, String requestContext) {
                                                 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);
     stage.setStageId(stageId);
     long now = System.currentTimeMillis();
     long now = System.currentTimeMillis();
     stage.addServerActionCommand(ServerAction.Command.FINALIZE_UPGRADE, Role.AMBARI_SERVER_ACTION,
     stage.addServerActionCommand(ServerAction.Command.FINALIZE_UPGRADE, Role.AMBARI_SERVER_ACTION,
@@ -898,7 +901,8 @@ public class TestActionScheduler {
     final List<Stage> stages = new ArrayList<Stage>();
     final List<Stage> stages = new ArrayList<Stage>();
 
 
     long now = System.currentTimeMillis();
     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.setStageId(1);
 
 
     addHostRoleExecutionCommand(now, stage, Role.SQOOP, Service.Type.SQOOP,
     addHostRoleExecutionCommand(now, stage, Role.SQOOP, Service.Type.SQOOP,
@@ -1083,7 +1087,7 @@ public class TestActionScheduler {
     final List<Stage> stages = new ArrayList<Stage>();
     final List<Stage> stages = new ArrayList<Stage>();
 
 
     long now = System.currentTimeMillis();
     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.setStageId(1);
     stage.addHostRoleExecutionCommand("host1", Role.DATANODE, RoleCommand.UPGRADE,
     stage.addHostRoleExecutionCommand("host1", Role.DATANODE, RoleCommand.UPGRADE,
         new ServiceComponentHostUpgradeEvent(Role.DATANODE.toString(), "host1", now, "HDP-0.2"),
         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,
   private Stage getStageWithSingleTask(String hostname, String clusterName, Role role,
                                        RoleCommand roleCommand, Service.Type service, int taskId,
                                        RoleCommand roleCommand, Service.Type service, int taskId,
                                        int stageId, int requestId) {
                                        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.setStageId(stageId);
     stage.addHostRoleExecutionCommand(hostname, role, roleCommand,
     stage.addHostRoleExecutionCommand(hostname, role, roleCommand,
         new ServiceComponentHostUpgradeEvent(role.toString(), hostname, System.currentTimeMillis(), "HDP-0.2"),
         new ServiceComponentHostUpgradeEvent(role.toString(), hostname, System.currentTimeMillis(), "HDP-0.2"),
@@ -1395,7 +1399,7 @@ public class TestActionScheduler {
     when(serviceObj.getCluster()).thenReturn(oneClusterMock);
     when(serviceObj.getCluster()).thenReturn(oneClusterMock);
 
 
     final List<Stage> stages = new ArrayList<Stage>();
     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);
             CLUSTER_HOST_INFO);
     addInstallTaskToStage(stage1, hostname1, "cluster1", Role.HBASE_MASTER,
     addInstallTaskToStage(stage1, hostname1, "cluster1", Role.HBASE_MASTER,
             RoleCommand.INSTALL, Service.Type.HBASE, 1);
             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
   @Test
   public void testGetRequestContext() {
   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("My Context", stage.getRequestContext());
     assertEquals(CLUSTER_HOST_INFO, stage.getClusterHostInfo());
     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) {
   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);
     s.setStageId(stageId);
     String filename = null;
     String filename = null;
     s.addHostRoleExecutionCommand(DummyHostname1, Role.HBASE_MASTER,
     s.addHostRoleExecutionCommand(DummyHostname1, Role.HBASE_MASTER,
@@ -977,7 +977,7 @@ public class TestHeartbeatHandler {
       getServiceComponent(DATANODE).getServiceComponentHost(DummyHostname1);
       getServiceComponent(DATANODE).getServiceComponentHost(DummyHostname1);
     serviceComponentHost1.setState(State.INSTALLING);
     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");
       "clusterHostInfo");
     s.setStageId(1);
     s.setStageId(1);
     s.addHostRoleExecutionCommand(DummyHostname1, Role.DATANODE, RoleCommand.INSTALL,
     s.addHostRoleExecutionCommand(DummyHostname1, Role.DATANODE, RoleCommand.INSTALL,
@@ -1493,7 +1493,7 @@ public class TestHeartbeatHandler {
     serviceComponentHost1.setDesiredStackVersion(stack130);
     serviceComponentHost1.setDesiredStackVersion(stack130);
     serviceComponentHost2.setStackVersion(stack122);
     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");
       "clusterHostInfo");
     s.setStageId(stageId);
     s.setStageId(stageId);
     s.addHostRoleExecutionCommand(DummyHostname1, Role.DATANODE, RoleCommand.UPGRADE,
     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.HostDAO;
 import org.apache.ambari.server.orm.dao.RoleDAO;
 import org.apache.ambari.server.orm.dao.RoleDAO;
 import org.apache.ambari.server.orm.entities.ExecutionCommandEntity;
 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.orm.entities.RoleEntity;
 import org.apache.ambari.server.security.authorization.Users;
 import org.apache.ambari.server.security.authorization.Users;
 import org.apache.ambari.server.serveraction.ServerAction;
 import org.apache.ambari.server.serveraction.ServerAction;
@@ -219,12 +220,17 @@ public class AmbariManagementControllerTest {
     host.setHostAttributes(hostAttributes);
     host.setHostAttributes(hostAttributes);
   }
   }
   
   
+  private void addHost(String hostname) throws AmbariException {
+    addHost(hostname, null);
+  }
+  
   private void addHost(String hostname, String clusterName) throws AmbariException {
   private void addHost(String hostname, String clusterName) throws AmbariException {
     clusters.addHost(hostname);
     clusters.addHost(hostname);
     setOsFamily(clusters.getHost(hostname), "redhat", "6.3");
     setOsFamily(clusters.getHost(hostname), "redhat", "6.3");
     clusters.getHost(hostname).setState(HostState.HEALTHY);
     clusters.getHost(hostname).setState(HostState.HEALTHY);
     clusters.getHost(hostname).persist();
     clusters.getHost(hostname).persist();
-    clusters.mapHostToCluster(hostname, clusterName);
+    if (null != clusterName)
+      clusters.mapHostToCluster(hostname, clusterName);
   }
   }
 
 
   private void createCluster(String clusterName) throws AmbariException {
   private void createCluster(String clusterName) throws AmbariException {
@@ -7716,7 +7722,7 @@ public class AmbariManagementControllerTest {
 
 
 
 
     List<Stage> stages = new ArrayList<Stage>();
     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).setStageId(1);
     stages.get(0).addHostRoleExecutionCommand(hostName1, Role.HBASE_MASTER,
     stages.get(0).addHostRoleExecutionCommand(hostName1, Role.HBASE_MASTER,
             RoleCommand.START,
             RoleCommand.START,
@@ -7724,14 +7730,14 @@ public class AmbariManagementControllerTest {
                     hostName1, System.currentTimeMillis()),
                     hostName1, System.currentTimeMillis()),
             clusterName, "HBASE");
             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).setStageId(2);
     stages.get(1).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
     stages.get(1).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
             RoleCommand.START,
             new ServiceComponentHostStartEvent(Role.HBASE_CLIENT.toString(),
             new ServiceComponentHostStartEvent(Role.HBASE_CLIENT.toString(),
                     hostName1, System.currentTimeMillis()), clusterName, "HBASE");
                     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).setStageId(3);
     stages.get(2).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
     stages.get(2).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
             RoleCommand.START,
@@ -7742,14 +7748,14 @@ public class AmbariManagementControllerTest {
     actionDB.persistActions(request);
     actionDB.persistActions(request);
 
 
     stages.clear();
     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).setStageId(4);
     stages.get(0).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
     stages.get(0).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
             RoleCommand.START,
             new ServiceComponentHostStartEvent(Role.HBASE_CLIENT.toString(),
             new ServiceComponentHostStartEvent(Role.HBASE_CLIENT.toString(),
                     hostName1, System.currentTimeMillis()), clusterName, "HBASE");
                     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).setStageId(5);
     stages.get(1).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
     stages.get(1).addHostRoleExecutionCommand(hostName1, Role.HBASE_CLIENT,
             RoleCommand.START,
             RoleCommand.START,
@@ -9972,6 +9978,92 @@ public class AmbariManagementControllerTest {
       // expected
       // 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;
 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.AmbariException;
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.actionmanager.HostRoleCommand;
 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.Predicate;
 import org.apache.ambari.server.controller.spi.Request;
 import org.apache.ambari.server.controller.spi.Request;
 import org.apache.ambari.server.controller.spi.Resource;
 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.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.PredicateBuilder;
 import org.apache.ambari.server.controller.utilities.PropertyHelper;
 import org.apache.ambari.server.controller.utilities.PropertyHelper;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Cluster;
@@ -41,24 +59,6 @@ import org.apache.ambari.server.state.Clusters;
 import org.easymock.Capture;
 import org.easymock.Capture;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 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.
  * RequestResourceProvider tests.
@@ -1062,5 +1062,121 @@ public class RequestResourceProviderTest {
     Assert.assertEquals(level.getHostName(), host_id);
     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);    
+  }
 
 
 }
 }