|
@@ -28,7 +28,9 @@ import java.util.Map;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
@@ -48,6 +50,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.apache.log4j.Level;
|
|
@@ -478,6 +481,124 @@ public class TestApplicationCleanup {
|
|
|
rm1.stop();
|
|
|
}
|
|
|
|
|
|
+ // The test verifies processing of NMContainerStatuses which are sent during
|
|
|
+ // NM registration.
|
|
|
+ // 1. Start the cluster-RM,NM,Submit app with 1024MB,Launch & register AM
|
|
|
+ // 2. AM sends ResourceRequest for 1 container with memory 2048MB.
|
|
|
+ // 3. Verify for number of container allocated by RM
|
|
|
+ // 4. Verify Memory Usage by cluster, it should be 3072. AM memory + requested
|
|
|
+ // memory. 1024 + 2048=3072
|
|
|
+ // 5. Re-register NM by sending completed container status
|
|
|
+ // 6. Verify for Memory Used, it should be 1024
|
|
|
+ // 7. Send AM heatbeat to RM. Allocated response should contain completed
|
|
|
+ // container.
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testProcessingNMContainerStatusesOnNMRestart() throws Exception {
|
|
|
+ conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
|
|
|
+ MemoryRMStateStore memStore = new MemoryRMStateStore();
|
|
|
+ memStore.init(conf);
|
|
|
+
|
|
|
+ // 1. Start the cluster-RM,NM,Submit app with 1024MB,Launch & register AM
|
|
|
+ MockRM rm1 = new MockRM(conf, memStore);
|
|
|
+ rm1.start();
|
|
|
+ int nmMemory = 8192;
|
|
|
+ int amMemory = 1024;
|
|
|
+ int containerMemory = 2048;
|
|
|
+ MockNM nm1 =
|
|
|
+ new MockNM("127.0.0.1:1234", nmMemory, rm1.getResourceTrackerService());
|
|
|
+ nm1.registerNode();
|
|
|
+
|
|
|
+ RMApp app0 = rm1.submitApp(amMemory);
|
|
|
+ MockAM am0 = MockRM.launchAndRegisterAM(app0, rm1, nm1);
|
|
|
+
|
|
|
+ // 2. AM sends ResourceRequest for 1 container with memory 2048MB.
|
|
|
+ int noOfContainers = 1;
|
|
|
+ List<Container> allocateContainers =
|
|
|
+ am0.allocateAndWaitForContainers(noOfContainers, containerMemory, nm1);
|
|
|
+
|
|
|
+ // 3. Verify for number of container allocated by RM
|
|
|
+ Assert.assertEquals(noOfContainers, allocateContainers.size());
|
|
|
+ Container container = allocateContainers.get(0);
|
|
|
+
|
|
|
+ nm1.nodeHeartbeat(am0.getApplicationAttemptId(), 1, ContainerState.RUNNING);
|
|
|
+ nm1.nodeHeartbeat(am0.getApplicationAttemptId(), container.getId()
|
|
|
+ .getContainerId(), ContainerState.RUNNING);
|
|
|
+
|
|
|
+ rm1.waitForState(app0.getApplicationId(), RMAppState.RUNNING);
|
|
|
+
|
|
|
+ // 4. Verify Memory Usage by cluster, it should be 3072. AM memory +
|
|
|
+ // requested memory. 1024 + 2048=3072
|
|
|
+ ResourceScheduler rs = rm1.getRMContext().getScheduler();
|
|
|
+ int allocatedMB = rs.getRootQueueMetrics().getAllocatedMB();
|
|
|
+ Assert.assertEquals(amMemory + containerMemory, allocatedMB);
|
|
|
+
|
|
|
+ // 5. Re-register NM by sending completed container status
|
|
|
+ List<NMContainerStatus> nMContainerStatusForApp =
|
|
|
+ createNMContainerStatusForApp(am0);
|
|
|
+ nm1.registerNode(nMContainerStatusForApp,
|
|
|
+ Arrays.asList(app0.getApplicationId()));
|
|
|
+
|
|
|
+ waitForClusterMemory(nm1, rs, amMemory);
|
|
|
+
|
|
|
+ // 6. Verify for Memory Used, it should be 1024
|
|
|
+ Assert.assertEquals(amMemory, rs.getRootQueueMetrics().getAllocatedMB());
|
|
|
+
|
|
|
+ // 7. Send AM heatbeat to RM. Allocated response should contain completed
|
|
|
+ // container
|
|
|
+ AllocateRequest req =
|
|
|
+ AllocateRequest.newInstance(0, 0F, new ArrayList<ResourceRequest>(),
|
|
|
+ new ArrayList<ContainerId>(), null);
|
|
|
+ AllocateResponse allocate = am0.allocate(req);
|
|
|
+ List<ContainerStatus> completedContainersStatuses =
|
|
|
+ allocate.getCompletedContainersStatuses();
|
|
|
+ Assert.assertEquals(noOfContainers, completedContainersStatuses.size());
|
|
|
+
|
|
|
+ // Application clean up should happen Cluster memory used is 0
|
|
|
+ nm1.nodeHeartbeat(am0.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
|
|
|
+ waitForClusterMemory(nm1, rs, 0);
|
|
|
+
|
|
|
+ rm1.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void waitForClusterMemory(MockNM nm1, ResourceScheduler rs,
|
|
|
+ int clusterMemory) throws Exception, InterruptedException {
|
|
|
+ int counter = 0;
|
|
|
+ while (rs.getRootQueueMetrics().getAllocatedMB() != clusterMemory) {
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
+
|
|
|
+ Thread.sleep(100);
|
|
|
+ if (counter++ == 50) {
|
|
|
+ Assert.fail("Wait for cluster memory is timed out.Expected="
|
|
|
+ + clusterMemory + " Actual="
|
|
|
+ + rs.getRootQueueMetrics().getAllocatedMB());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public static List<NMContainerStatus> createNMContainerStatusForApp(MockAM am) {
|
|
|
+ List<NMContainerStatus> list = new ArrayList<NMContainerStatus>();
|
|
|
+ NMContainerStatus amContainer =
|
|
|
+ createNMContainerStatus(am.getApplicationAttemptId(), 1,
|
|
|
+ ContainerState.RUNNING, 1024);
|
|
|
+ NMContainerStatus completedContainer =
|
|
|
+ createNMContainerStatus(am.getApplicationAttemptId(), 2,
|
|
|
+ ContainerState.COMPLETE, 2048);
|
|
|
+ list.add(amContainer);
|
|
|
+ list.add(completedContainer);
|
|
|
+ return list;
|
|
|
+ }
|
|
|
+
|
|
|
+ public static NMContainerStatus createNMContainerStatus(
|
|
|
+ ApplicationAttemptId appAttemptId, int id, ContainerState containerState,
|
|
|
+ int memory) {
|
|
|
+ ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
|
|
|
+ NMContainerStatus containerReport =
|
|
|
+ NMContainerStatus.newInstance(containerId, containerState,
|
|
|
+ Resource.newInstance(memory, 1), "recover container", 0,
|
|
|
+ Priority.newInstance(0), 0);
|
|
|
+ return containerReport;
|
|
|
+ }
|
|
|
+
|
|
|
public static void main(String[] args) throws Exception {
|
|
|
TestApplicationCleanup t = new TestApplicationCleanup();
|
|
|
t.testAppCleanup();
|