Przeglądaj źródła

YARN-879. Fixed tests w.r.t o.a.h.y.server.resourcemanager.Application. Contributed by Junping Du.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1530902 13f79535-47bb-0310-9956-ffa450edef68
Devarajulu K 11 lat temu
rodzic
commit
22b332ff46

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -95,6 +95,9 @@ Release 2.2.1 - UNRELEASED
     so that clients don't need to do scheme-mangling. (Omkar Vinit Joshi via
     vinodkv)
 
+    YARN-879. Fixed tests w.r.t o.a.h.y.server.resourcemanager.Application.
+    (Junping Du via devaraj)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 41 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java

@@ -34,6 +34,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
@@ -47,11 +49,16 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.Task.State;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
+import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 @Private
@@ -89,16 +96,23 @@ public class Application {
   
   Resource used = recordFactory.newRecordInstance(Resource.class);
   
-  public Application(String user, ResourceManager resourceManager) {
+  public Application(String user, ResourceManager resourceManager) 
+      throws YarnException {
     this(user, "default", resourceManager);
   }
   
-  public Application(String user, String queue, ResourceManager resourceManager) {
+  public Application(String user, String queue, ResourceManager resourceManager) 
+      throws YarnException {
     this.user = user;
     this.queue = queue;
     this.resourceManager = resourceManager;
-    this.applicationId =
-      this.resourceManager.getClientRMService().getNewApplicationId();
+    // register an application
+    GetNewApplicationRequest request =
+            Records.newRecord(GetNewApplicationRequest.class);
+    GetNewApplicationResponse newApp = 
+        this.resourceManager.getClientRMService().getNewApplication(request);
+    this.applicationId = newApp.getApplicationId();
+  
     this.applicationAttemptId =
         ApplicationAttemptId.newInstance(this.applicationId,
           this.numAttempts.getAndIncrement());
@@ -115,6 +129,10 @@ public class Application {
   public ApplicationId getApplicationId() {
     return applicationId;
   }
+  
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return applicationAttemptId;
+  }
 
   public static String resolve(String hostName) {
     return NetworkTopology.DEFAULT_RACK;
@@ -132,10 +150,25 @@ public class Application {
     ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     context.setApplicationId(this.applicationId);
     context.setQueue(this.queue);
+    
+    // Set up the container launch context for the application master
+    ContainerLaunchContext amContainer
+        = Records.newRecord(ContainerLaunchContext.class);
+    context.setAMContainerSpec(amContainer);
+    context.setResource(Resources.createResource(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB));
+    
     SubmitApplicationRequest request = recordFactory
         .newRecordInstance(SubmitApplicationRequest.class);
     request.setApplicationSubmissionContext(context);
+    final ResourceScheduler scheduler = resourceManager.getResourceScheduler();
+    
     resourceManager.getClientRMService().submitApplication(request);
+    
+    // Notify scheduler
+    AppAddedSchedulerEvent appAddedEvent1 = new AppAddedSchedulerEvent(
+            this.applicationAttemptId, this.queue, this.user);
+    scheduler.handle(appAddedEvent1);
   }
   
   public synchronized void addResourceRequestSpec(
@@ -267,17 +300,13 @@ public class Application {
     }
     
     // Get resources from the ResourceManager
-    resourceManager.getResourceScheduler().allocate(applicationAttemptId,
-        new ArrayList<ResourceRequest>(ask), new ArrayList<ContainerId>(), null, null);
+    Allocation allocation = resourceManager.getResourceScheduler().allocate(
+        applicationAttemptId, new ArrayList<ResourceRequest>(ask),
+        new ArrayList<ContainerId>(), null, null);
     System.out.println("-=======" + applicationAttemptId);
     System.out.println("----------" + resourceManager.getRMContext().getRMApps()
         .get(applicationId).getRMAppAttempt(applicationAttemptId));
-    
-     List<Container> containers = null;
-     // TODO: Fix
-//       resourceManager.getRMContext().getRMApps()
-//        .get(applicationId).getRMAppAttempt(applicationAttemptId)
-//        .pullNewlyAllocatedContainers();
+    List<Container> containers = allocation.getContainers();
 
     // Clear state for next interaction with ResourceManager
     ask.clear();

+ 12 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java

@@ -56,7 +56,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -71,28 +70,27 @@ public class NodeManager implements ContainerManagementProtocol {
   final private String rackName;
   final private NodeId nodeId;
   final private Resource capability;
+  final private ResourceManager resourceManager;
   Resource available = recordFactory.newRecordInstance(Resource.class);
   Resource used = recordFactory.newRecordInstance(Resource.class);
 
   final ResourceTrackerService resourceTrackerService;
-  final FiCaSchedulerNode schedulerNode;
   final Map<ApplicationId, List<Container>> containers = 
     new HashMap<ApplicationId, List<Container>>();
   
   final Map<Container, ContainerStatus> containerStatusMap =
       new HashMap<Container, ContainerStatus>();
-
+  
   public NodeManager(String hostName, int containerManagerPort, int httpPort,
       String rackName, Resource capability,
-      ResourceTrackerService resourceTrackerService, RMContext rmContext)
+      ResourceManager resourceManager)
       throws IOException, YarnException {
     this.containerManagerAddress = hostName + ":" + containerManagerPort;
     this.nodeHttpAddress = hostName + ":" + httpPort;
     this.rackName = rackName;
-    this.resourceTrackerService = resourceTrackerService;
+    this.resourceTrackerService = resourceManager.getResourceTrackerService();
     this.capability = capability;
     Resources.addTo(available, capability);
-
     this.nodeId = NodeId.newInstance(hostName, containerManagerPort);
     RegisterNodeManagerRequest request = recordFactory
         .newRecordInstance(RegisterNodeManagerRequest.class);
@@ -101,14 +99,8 @@ public class NodeManager implements ContainerManagementProtocol {
     request.setNodeId(this.nodeId);
     request.setNMVersion(YarnVersionInfo.getVersion());
     resourceTrackerService.registerNodeManager(request);
-    this.schedulerNode = new FiCaSchedulerNode(rmContext.getRMNodes().get(
-        this.nodeId), false);
-   
-    // Sanity check
-    Assert.assertEquals(capability.getMemory(), 
-       schedulerNode.getAvailableResource().getMemory());
-    Assert.assertEquals(capability.getVirtualCores(), 
-        schedulerNode.getAvailableResource().getVirtualCores());
+    this.resourceManager = resourceManager;
+    resourceManager.getResourceScheduler().getNodeReport(this.nodeId);
   }
   
   public String getHostName() {
@@ -220,9 +212,11 @@ public class NodeManager implements ContainerManagementProtocol {
   synchronized public void checkResourceUsage() {
     LOG.info("Checking resource usage for " + containerManagerAddress);
     Assert.assertEquals(available.getMemory(), 
-        schedulerNode.getAvailableResource().getMemory());
+        resourceManager.getResourceScheduler().getNodeReport(
+            this.nodeId).getAvailableResource().getMemory());
     Assert.assertEquals(used.getMemory(), 
-        schedulerNode.getUsedResource().getMemory());
+        resourceManager.getResourceScheduler().getNodeReport(
+            this.nodeId).getUsedResource().getMemory());
   }
   
   @Override
@@ -232,9 +226,9 @@ public class NodeManager implements ContainerManagementProtocol {
       String applicationId =
           String.valueOf(containerID.getApplicationAttemptId()
             .getApplicationId().getId());
-
       // Mark the container as COMPLETE
-      List<Container> applicationContainers = containers.get(applicationId);
+      List<Container> applicationContainers = containers.get(containerID.getApplicationAttemptId()
+              .getApplicationId());
       for (Container c : applicationContainers) {
         if (c.getId().compareTo(containerID) == 0) {
           ContainerStatus containerStatus = containerStatusMap.get(c);

+ 34 - 23
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java

@@ -34,7 +34,11 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.Before;
@@ -62,13 +66,18 @@ public class TestResourceManager {
       registerNode(String hostName, int containerManagerPort, int httpPort,
           String rackName, Resource capability) throws IOException,
           YarnException {
-    return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
-        hostName, containerManagerPort, httpPort, rackName, capability,
-        resourceManager.getResourceTrackerService(), resourceManager
-            .getRMContext());
+    org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm = 
+        new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
+            hostName, containerManagerPort, httpPort, rackName, capability,
+            resourceManager);
+    NodeAddedSchedulerEvent nodeAddEvent1 = 
+        new NodeAddedSchedulerEvent(resourceManager.getRMContext()
+            .getRMNodes().get(nm.getNodeId()));
+    resourceManager.getResourceScheduler().handle(nodeAddEvent1);
+    return nm;
   }
 
-//  @Test
+  @Test
   public void testResourceAllocation() throws IOException,
       YarnException {
     LOG.info("--- START: testResourceAllocation ---");
@@ -80,14 +89,12 @@ public class TestResourceManager {
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm1 = 
       registerNode(host1, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
           Resources.createResource(memory, 1));
-    nm1.heartbeat();
     
     // Register node2
     String host2 = "host2";
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm2 = 
       registerNode(host2, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
           Resources.createResource(memory/2, 1));
-    nm2.heartbeat();
 
     // Submit an application
     Application application = new Application("user1", resourceManager);
@@ -105,23 +112,22 @@ public class TestResourceManager {
     
     Task t1 = new Task(application, priority1, new String[] {host1, host2});
     application.addTask(t1);
-        
+    
     final int memory2 = 2048;
     Resource capability2 = Resources.createResource(memory2, 1);
     Priority priority0 = 
-      org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(0); // higher
+        org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(0); // higher
     application.addResourceRequestSpec(priority0, capability2);
     
     // Send resource requests to the scheduler
     application.schedule();
-    
-    // Send a heartbeat to kick the tires on the Scheduler
-    nm1.heartbeat();
+
+   // Send a heartbeat to kick the tires on the Scheduler
+    nodeUpdate(nm1);
     
     // Get allocations from the scheduler
     application.schedule();
     
-    nm1.heartbeat();
     checkResourceUsage(nm1, nm2);
     
     LOG.info("Adding new tasks...");
@@ -137,18 +143,13 @@ public class TestResourceManager {
     checkResourceUsage(nm1, nm2);
     
     // Send a heartbeat to kick the tires on the Scheduler
-    LOG.info("Sending hb from host2");
-    nm2.heartbeat();
-    
-    LOG.info("Sending hb from host1");
-    nm1.heartbeat();
+    nodeUpdate(nm2);
+    nodeUpdate(nm1);
     
     // Get allocations from the scheduler
     LOG.info("Trying to allocate...");
     application.schedule();
 
-    nm1.heartbeat();
-    nm2.heartbeat();
     checkResourceUsage(nm1, nm2);
     
     // Complete tasks
@@ -157,13 +158,23 @@ public class TestResourceManager {
     application.finishTask(t2);
     application.finishTask(t3);
     
-    // Send heartbeat
-    nm1.heartbeat();
-    nm2.heartbeat();
+    // Notify scheduler application is finished.
+    AppRemovedSchedulerEvent appRemovedEvent1 = new AppRemovedSchedulerEvent(
+        application.getApplicationAttemptId(), RMAppAttemptState.FINISHED);
+    resourceManager.getResourceScheduler().handle(appRemovedEvent1);
+    
     checkResourceUsage(nm1, nm2);
     
     LOG.info("--- END: testResourceAllocation ---");
   }
+
+  private void nodeUpdate(
+      org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm1) {
+    RMNode node = resourceManager.getRMContext().getRMNodes().get(nm1.getNodeId());
+    // Send a heartbeat to kick the tires on the Scheduler
+    NodeUpdateSchedulerEvent nodeUpdate = new NodeUpdateSchedulerEvent(node);
+    resourceManager.getResourceScheduler().handle(nodeUpdate);
+  }
   
   @Test
   public void testNodeHealthReportIsNotNull() throws Exception{

+ 36 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
@@ -111,6 +112,8 @@ public class TestCapacityScheduler {
     conf.setClass(YarnConfiguration.RM_SCHEDULER, 
         CapacityScheduler.class, ResourceScheduler.class);
     resourceManager.init(conf);
+    resourceManager.getRMContainerTokenSecretManager().rollMasterKey();
+    resourceManager.getRMNMTokenSecretManager().rollMasterKey();
     ((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
   }
 
@@ -156,13 +159,18 @@ public class TestCapacityScheduler {
       registerNode(String hostName, int containerManagerPort, int httpPort,
           String rackName, Resource capability)
           throws IOException, YarnException {
-    return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
-        hostName, containerManagerPort, httpPort, rackName, capability,
-        resourceManager.getResourceTrackerService(), resourceManager
-            .getRMContext());
-  }  
+    org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm =
+        new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
+            hostName, containerManagerPort, httpPort, rackName, capability,
+            resourceManager);
+    NodeAddedSchedulerEvent nodeAddEvent1 = 
+        new NodeAddedSchedulerEvent(resourceManager.getRMContext()
+            .getRMNodes().get(nm.getNodeId()));
+    resourceManager.getResourceScheduler().handle(nodeAddEvent1);
+    return nm;
+  }
 
-//  @Test
+  @Test
   public void testCapacityScheduler() throws Exception {
 
     LOG.info("--- START: testCapacityScheduler ---");
@@ -172,14 +180,12 @@ public class TestCapacityScheduler {
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_0 = 
       registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
           Resources.createResource(4 * GB, 1));
-    nm_0.heartbeat();
     
     // Register node2
     String host_1 = "host_1";
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_1 = 
       registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
           Resources.createResource(2 * GB, 1));
-    nm_1.heartbeat();
 
     // ResourceRequest priorities
     Priority priority_0 = 
@@ -227,9 +233,13 @@ public class TestCapacityScheduler {
 
     // Send a heartbeat to kick the tires on the Scheduler
     LOG.info("Kick!");
-    nm_0.heartbeat();             // task_0_0 and task_1_0 allocated, used=4G
-    nm_1.heartbeat();             // nothing allocated
-
+    
+    // task_0_0 and task_1_0 allocated, used=4G
+    nodeUpdate(nm_0);
+    
+    // nothing allocated
+    nodeUpdate(nm_1);
+    
     // Get allocations from the scheduler
     application_0.schedule();     // task_0_0 
     checkApplicationResourceUsage(1 * GB, application_0);
@@ -237,9 +247,6 @@ public class TestCapacityScheduler {
     application_1.schedule();     // task_1_0
     checkApplicationResourceUsage(3 * GB, application_1);
     
-    nm_0.heartbeat();
-    nm_1.heartbeat();
-    
     checkNodeResourceUsage(4*GB, nm_0);  // task_0_0 (1G) and task_1_0 (3G)
     checkNodeResourceUsage(0*GB, nm_1);  // no tasks, 2G available
 
@@ -259,10 +266,12 @@ public class TestCapacityScheduler {
 
     // Send a heartbeat to kick the tires on the Scheduler
     LOG.info("Sending hb from " + nm_0.getHostName());
-    nm_0.heartbeat();                   // nothing new, used=4G
+    // nothing new, used=4G
+    nodeUpdate(nm_0);
     
     LOG.info("Sending hb from " + nm_1.getHostName());
-    nm_1.heartbeat();                   // task_0_3, used=2G
+    // task_0_1 is prefer as locality, used=2G
+    nodeUpdate(nm_1);
     
     // Get allocations from the scheduler
     LOG.info("Trying to allocate...");
@@ -272,13 +281,22 @@ public class TestCapacityScheduler {
     application_1.schedule();
     checkApplicationResourceUsage(5 * GB, application_1);
     
-    nm_0.heartbeat();
-    nm_1.heartbeat();
+    nodeUpdate(nm_0);
+    nodeUpdate(nm_1);
+    
     checkNodeResourceUsage(4*GB, nm_0);
     checkNodeResourceUsage(2*GB, nm_1);
 
     LOG.info("--- END: testCapacityScheduler ---");
   }
+
+  private void nodeUpdate(
+      org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm) {
+    RMNode node = resourceManager.getRMContext().getRMNodes().get(nm.getNodeId());
+    // Send a heartbeat to kick the tires on the Scheduler
+    NodeUpdateSchedulerEvent nodeUpdate = new NodeUpdateSchedulerEvent(node);
+    resourceManager.getResourceScheduler().handle(nodeUpdate);
+  }
   
   private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
     

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

@@ -96,8 +96,7 @@ public class TestFifoScheduler {
           YarnException {
     return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
         hostName, containerManagerPort, nmHttpPort, rackName, capability,
-        resourceManager.getResourceTrackerService(), resourceManager
-            .getRMContext());
+        resourceManager);
   }
   
   private ApplicationAttemptId createAppAttemptId(int appId, int attemptId) {