You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-commits@hadoop.apache.org by sz...@apache.org on 2014/01/15 07:06:35 UTC
svn commit: r1558303 [4/5] - in
/hadoop/common/branches/HDFS-5535/hadoop-yarn-project: ./ hadoop-yarn/
hadoop-yarn/hadoop-yarn-api/
hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/
hadoop-yarn/hadoop-yarn-api/src/ma...
Modified: hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java?rev=1558303&r1=1558302&r2=1558303&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java (original)
+++ hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java Wed Jan 15 06:06:31 2014
@@ -18,49 +18,31 @@
package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicInteger;
import junit.framework.Assert;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.QueueInfo;
-import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
-import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
-import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
-import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.After;
-import org.junit.Before;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+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.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.junit.Test;
/**
@@ -68,238 +50,186 @@ import org.junit.Test;
*
*/
public class TestAMRestart {
-// private static final Log LOG = LogFactory.getLog(TestAMRestart.class);
-// ApplicationsManagerImpl appImpl;
-// RMContext asmContext = new RMContextImpl(new MemStore());
-// ApplicationTokenSecretManager appTokenSecretManager =
-// new ApplicationTokenSecretManager();
-// DummyResourceScheduler scheduler;
-// private ClientRMService clientRMService;
-// int count = 0;
-// ApplicationId appID;
-// final int maxFailures = 3;
-// AtomicInteger launchNotify = new AtomicInteger();
-// AtomicInteger schedulerNotify = new AtomicInteger();
-// volatile boolean stop = false;
-// int schedulerAddApplication = 0;
-// int schedulerRemoveApplication = 0;
-// int launcherLaunchCalled = 0;
-// int launcherCleanupCalled = 0;
-// private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-//
-// private class ExtApplicationsManagerImpl extends ApplicationsManagerImpl {
-// public ExtApplicationsManagerImpl(
-// ApplicationTokenSecretManager applicationTokenSecretManager,
-// YarnScheduler scheduler, RMContext asmContext) {
-// super(applicationTokenSecretManager, scheduler, asmContext);
-// }
-//
-// @Override
-// public EventHandler<ASMEvent<AMLauncherEventType>> createNewApplicationMasterLauncher(
-// ApplicationTokenSecretManager tokenSecretManager) {
-// return new DummyAMLauncher();
-// }
-// }
-//
-// private class DummyAMLauncher implements EventHandler<ASMEvent<AMLauncherEventType>> {
-//
-// public DummyAMLauncher() {
-// asmContext.getDispatcher().register(AMLauncherEventType.class, this);
-// new Thread() {
-// public void run() {
-// while (!stop) {
-// LOG.info("DEBUG -- waiting for launch");
-// synchronized(launchNotify) {
-// while (launchNotify.get() == 0) {
-// try {
-// launchNotify.wait();
-// } catch (InterruptedException e) {
-// }
-// }
-// asmContext.getDispatcher().getEventHandler().handle(
-// new ApplicationEvent(
-// ApplicationEventType.LAUNCHED, appID));
-// launchNotify.addAndGet(-1);
-// }
-// }
-// }
-// }.start();
-// }
-//
-// @Override
-// public void handle(ASMEvent<AMLauncherEventType> event) {
-// switch (event.getType()) {
-// case CLEANUP:
-// launcherCleanupCalled++;
-// break;
-// case LAUNCH:
-// LOG.info("DEBUG -- launching");
-// launcherLaunchCalled++;
-// synchronized (launchNotify) {
-// launchNotify.addAndGet(1);
-// launchNotify.notify();
-// }
-// break;
-// default:
-// break;
-// }
-// }
-// }
-//
-// private class DummyResourceScheduler implements ResourceScheduler {
-//
-// @Override
-// public void removeNode(RMNode node) {
-// }
-//
-// @Override
-// public Allocation allocate(ApplicationId applicationId,
-// List<ResourceRequest> ask, List<Container> release) throws IOException {
-// Container container = recordFactory.newRecordInstance(Container.class);
-// container.setContainerToken(recordFactory.newRecordInstance(ContainerToken.class));
-// container.setNodeId(recordFactory.newRecordInstance(NodeId.class));
-// container.setContainerManagerAddress("localhost");
-// container.setNodeHttpAddress("localhost:8042");
-// container.setId(recordFactory.newRecordInstance(ContainerId.class));
-// container.getId().setAppId(appID);
-// container.getId().setId(count);
-// count++;
-// return new Allocation(Arrays.asList(container), Resources.none());
-// }
-//
-// @Override
-// public void handle(ASMEvent<ApplicationTrackerEventType> event) {
-// switch (event.getType()) {
-// case ADD:
-// schedulerAddApplication++;
-// break;
-// case EXPIRE:
-// schedulerRemoveApplication++;
-// LOG.info("REMOVING app : " + schedulerRemoveApplication);
-// if (schedulerRemoveApplication == maxFailures) {
-// synchronized (schedulerNotify) {
-// schedulerNotify.addAndGet(1);
-// schedulerNotify.notify();
-// }
-// }
-// break;
-// default:
-// break;
-// }
-// }
-//
-// @Override
-// public QueueInfo getQueueInfo(String queueName,
-// boolean includeChildQueues,
-// boolean recursive) throws IOException {
-// return null;
-// }
-// @Override
-// public List<QueueUserACLInfo> getQueueUserAclInfo() {
-// return null;
-// }
-// @Override
-// public void addApplication(ApplicationId applicationId,
-// ApplicationMaster master, String user, String queue, Priority priority,
-// ApplicationStore store)
-// throws IOException {
-// }
-// @Override
-// public void addNode(RMNode nodeInfo) {
-// }
-// @Override
-// public void recover(RMState state) throws Exception {
-// }
-// @Override
-// public void reinitialize(Configuration conf,
-// ContainerTokenSecretManager secretManager, RMContext rmContext)
-// throws IOException {
-// }
-//
-// @Override
-// public void nodeUpdate(RMNode nodeInfo,
-// Map<String, List<Container>> containers) {
-// }
-//
-// @Override
-// public Resource getMaximumResourceCapability() {
-// // TODO Auto-generated method stub
-// return null;
-// }
-//
-// @Override
-// public Resource getMinimumResourceCapability() {
-// // TODO Auto-generated method stub
-// return null;
-// }
-// }
-//
-// @Before
-// public void setUp() {
-//
-// asmContext.getDispatcher().register(ApplicationEventType.class,
-// new ResourceManager.ApplicationEventDispatcher(asmContext));
-//
-// appID = recordFactory.newRecordInstance(ApplicationId.class);
-// appID.setClusterTimestamp(System.currentTimeMillis());
-// appID.setId(1);
-// Configuration conf = new Configuration();
-// scheduler = new DummyResourceScheduler();
-// asmContext.getDispatcher().init(conf);
-// asmContext.getDispatcher().start();
-// asmContext.getDispatcher().register(ApplicationTrackerEventType.class, scheduler);
-// appImpl = new ExtApplicationsManagerImpl(appTokenSecretManager, scheduler, asmContext);
-//
-// conf.setLong(YarnConfiguration.AM_EXPIRY_INTERVAL, 1000L);
-// conf.setInt(RMConfig.AM_MAX_RETRIES, maxFailures);
-// appImpl.init(conf);
-// appImpl.start();
-//
-// this.clientRMService = new ClientRMService(asmContext, appImpl
-// .getAmLivelinessMonitor(), appImpl.getClientToAMSecretManager(),
-// scheduler);
-// this.clientRMService.init(conf);
-// }
-//
-// @After
-// public void tearDown() {
-// }
-//
-// private void waitForFailed(AppAttempt application, ApplicationState
-// finalState) throws Exception {
-// int count = 0;
-// while(application.getState() != finalState && count < 10) {
-// Thread.sleep(500);
-// count++;
-// }
-// Assert.assertEquals(finalState, application.getState());
-// }
-//
-// @Test
-// public void testAMRestart() throws Exception {
-// ApplicationSubmissionContext subContext = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
-// subContext.setApplicationId(appID);
-// subContext.setApplicationName("dummyApp");
-//// subContext.command = new ArrayList<String>();
-//// subContext.environment = new HashMap<String, String>();
-//// subContext.fsTokens = new ArrayList<String>();
-// subContext.setFsTokensTodo(ByteBuffer.wrap(new byte[0]));
-// SubmitApplicationRequest request = recordFactory
-// .newRecordInstance(SubmitApplicationRequest.class);
-// request.setApplicationSubmissionContext(subContext);
-// clientRMService.submitApplication(request);
-// AppAttempt application = asmContext.getApplications().get(appID);
-// synchronized (schedulerNotify) {
-// while(schedulerNotify.get() == 0) {
-// schedulerNotify.wait();
-// }
-// }
-// Assert.assertEquals(maxFailures, launcherCleanupCalled);
-// Assert.assertEquals(maxFailures, launcherLaunchCalled);
-// Assert.assertEquals(maxFailures, schedulerAddApplication);
-// Assert.assertEquals(maxFailures, schedulerRemoveApplication);
-// Assert.assertEquals(maxFailures, application.getFailedCount());
-// waitForFailed(application, ApplicationState.FAILED);
-// stop = true;
-// }
+
+ @Test
+ public void testAMRestartWithExistingContainers() throws Exception {
+ YarnConfiguration conf = new YarnConfiguration();
+ conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+
+ MockRM rm1 = new MockRM(conf);
+ rm1.start();
+ RMApp app1 =
+ rm1.submitApp(200, "name", "user",
+ new HashMap<ApplicationAccessType, String>(), false, "default", -1,
+ null, "MAPREDUCE", false, true);
+ MockNM nm1 =
+ new MockNM("127.0.0.1:1234", 10240, rm1.getResourceTrackerService());
+ nm1.registerNode();
+ MockNM nm2 =
+ new MockNM("127.0.0.1:2351", 4089, rm1.getResourceTrackerService());
+ nm2.registerNode();
+
+ MockAM am1 = MockRM.launchAM(app1, rm1, nm1);
+ int NUM_CONTAINERS = 3;
+ // allocate NUM_CONTAINERS containers
+ am1.allocate("127.0.0.1", 1024, NUM_CONTAINERS,
+ new ArrayList<ContainerId>());
+ nm1.nodeHeartbeat(true);
+
+ // wait for containers to be allocated.
+ List<Container> containers =
+ am1.allocate(new ArrayList<ResourceRequest>(),
+ new ArrayList<ContainerId>()).getAllocatedContainers();
+ while (containers.size() != NUM_CONTAINERS) {
+ nm1.nodeHeartbeat(true);
+ containers.addAll(am1.allocate(new ArrayList<ResourceRequest>(),
+ new ArrayList<ContainerId>()).getAllocatedContainers());
+ Thread.sleep(200);
+ }
+
+ // launch the 2nd container, for testing running container transferred.
+ nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING);
+ ContainerId containerId2 =
+ ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+ rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
+
+ // launch the 3rd container, for testing container allocated by previous
+ // attempt is completed by the next new attempt/
+ nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.RUNNING);
+ ContainerId containerId3 =
+ ContainerId.newInstance(am1.getApplicationAttemptId(), 3);
+ rm1.waitForState(nm1, containerId3, RMContainerState.RUNNING);
+
+ // 4th container still in AQUIRED state. for testing Acquired container is
+ // always killed.
+ ContainerId containerId4 =
+ ContainerId.newInstance(am1.getApplicationAttemptId(), 4);
+ rm1.waitForState(nm1, containerId4, RMContainerState.ACQUIRED);
+
+ // 5th container is in Allocated state. for testing allocated container is
+ // always killed.
+ am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
+ nm1.nodeHeartbeat(true);
+ ContainerId containerId5 =
+ ContainerId.newInstance(am1.getApplicationAttemptId(), 5);
+ rm1.waitForContainerAllocated(nm1, containerId5);
+ rm1.waitForState(nm1, containerId5, RMContainerState.ALLOCATED);
+
+ // 6th container is in Reserved state.
+ am1.allocate("127.0.0.1", 6000, 1, new ArrayList<ContainerId>());
+ ContainerId containerId6 =
+ ContainerId.newInstance(am1.getApplicationAttemptId(), 6);
+ nm1.nodeHeartbeat(true);
+ SchedulerApplicationAttempt schedulerAttempt =
+ ((CapacityScheduler) rm1.getResourceScheduler())
+ .getCurrentAttemptForContainer(containerId6);
+ while (schedulerAttempt.getReservedContainers().size() == 0) {
+ System.out.println("Waiting for container " + containerId6
+ + " to be reserved.");
+ nm1.nodeHeartbeat(true);
+ Thread.sleep(200);
+ }
+ // assert containerId6 is reserved.
+ Assert.assertEquals(containerId6, schedulerAttempt.getReservedContainers()
+ .get(0).getContainerId());
+
+ // fail the AM by sending CONTAINER_FINISHED event without registering.
+ nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
+ am1.waitForState(RMAppAttemptState.FAILED);
+
+ // wait for some time. previous AM's running containers should still remain
+ // in scheduler even though am failed
+ Thread.sleep(3000);
+ rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
+ // acquired/allocated containers are cleaned up.
+ Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId4));
+ Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId5));
+
+ // wait for app to start a new attempt.
+ rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
+ // assert this is a new AM.
+ ApplicationAttemptId newAttemptId =
+ app1.getCurrentAppAttempt().getAppAttemptId();
+ Assert.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId()));
+
+ // launch the new AM
+ RMAppAttempt attempt2 = app1.getCurrentAppAttempt();
+ nm1.nodeHeartbeat(true);
+ MockAM am2 = rm1.sendAMLaunched(attempt2.getAppAttemptId());
+ RegisterApplicationMasterResponse registerResponse =
+ am2.registerAppAttempt();
+
+ // Assert two containers are running: container2 and container3;
+ Assert.assertEquals(2, registerResponse.getContainersFromPreviousAttempt()
+ .size());
+ boolean containerId2Exists = false, containerId3Exists = false;
+ for (Container container : registerResponse
+ .getContainersFromPreviousAttempt()) {
+ if (container.getId().equals(containerId2)) {
+ containerId2Exists = true;
+ }
+ if (container.getId().equals(containerId3)) {
+ containerId3Exists = true;
+ }
+ }
+ Assert.assertTrue(containerId2Exists && containerId3Exists);
+ rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
+
+ // complete container by sending the container complete event which has earlier
+ // attempt's attemptId
+ nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.COMPLETE);
+ rm1.waitForState(nm1, containerId3, RMContainerState.COMPLETED);
+
+ // Even though the completed container containerId3 event was sent to the
+ // earlier failed attempt, new RMAppAttempt can also capture this container
+ // info.
+ // completed containerId4 is also transferred to the new attempt.
+ RMAppAttempt newAttempt =
+ app1.getRMAppAttempt(am2.getApplicationAttemptId());
+ // 4 containers finished, acquired/allocated/reserved/completed.
+ Assert.assertEquals(4, newAttempt.getJustFinishedContainers().size());
+ boolean container3Exists = false, container4Exists = false, container5Exists =
+ false, container6Exists = false;
+ for(ContainerStatus status : newAttempt.getJustFinishedContainers()) {
+ if(status.getContainerId().equals(containerId3)) {
+ // containerId3 is the container ran by previous attempt but finished by the
+ // new attempt.
+ container3Exists = true;
+ }
+ if (status.getContainerId().equals(containerId4)) {
+ // containerId4 is the Acquired Container killed by the previous attempt,
+ // it's now inside new attempt's finished container list.
+ container4Exists = true;
+ }
+ if (status.getContainerId().equals(containerId5)) {
+ // containerId5 is the Allocated container killed by previous failed attempt.
+ container5Exists = true;
+ }
+ if (status.getContainerId().equals(containerId6)) {
+ // containerId6 is the reserved container killed by previous failed attempt.
+ container6Exists = true;
+ }
+ }
+ Assert.assertTrue(container3Exists && container4Exists && container5Exists
+ && container6Exists);
+
+ // New SchedulerApplicationAttempt also has the containers info.
+ rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
+
+ // record the scheduler attempt for testing.
+ SchedulerApplicationAttempt schedulerNewAttempt =
+ ((CapacityScheduler) rm1.getResourceScheduler())
+ .getCurrentAttemptForContainer(containerId2);
+ // finish this application
+ MockRM.finishApplicationMaster(app1, rm1, nm1, am2);
+
+ // the 2nd attempt released the 1st attempt's running container, when the
+ // 2nd attempt finishes.
+ Assert.assertFalse(schedulerNewAttempt.getLiveContainers().contains(
+ containerId2));
+ // all 4 normal containers finished.
+ Assert.assertEquals(5, newAttempt.getJustFinishedContainers().size());
+
+ rm1.stop();
+ }
}
Modified: hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java?rev=1558303&r1=1558302&r2=1558303&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java (original)
+++ hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java Wed Jan 15 06:06:31 2014
@@ -460,7 +460,7 @@ public class TestRMAppTransitions {
LOG.info("--- START: testUnmanagedAppFailPath ---");
application = testCreateAppRunning(subContext);
RMAppEvent event = new RMAppFailedAttemptEvent(
- application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED, "");
+ application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED, "", false);
application.handle(event);
rmDispatcher.await();
RMAppAttempt appAttempt = application.getCurrentAppAttempt();
@@ -582,7 +582,7 @@ public class TestRMAppTransitions {
for (int i=1; i < maxAppAttempts; i++) {
RMAppEvent event =
new RMAppFailedAttemptEvent(application.getApplicationId(),
- RMAppEventType.ATTEMPT_FAILED, "");
+ RMAppEventType.ATTEMPT_FAILED, "", false);
application.handle(event);
assertAppState(RMAppState.ACCEPTED, application);
event =
@@ -598,7 +598,7 @@ public class TestRMAppTransitions {
String message = "Test fail";
RMAppEvent event =
new RMAppFailedAttemptEvent(application.getApplicationId(),
- RMAppEventType.ATTEMPT_FAILED, message);
+ RMAppEventType.ATTEMPT_FAILED, message, false);
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
@@ -655,7 +655,7 @@ public class TestRMAppTransitions {
for (int i=1; i<maxAppAttempts; i++) {
RMAppEvent event =
new RMAppFailedAttemptEvent(application.getApplicationId(),
- RMAppEventType.ATTEMPT_FAILED, "");
+ RMAppEventType.ATTEMPT_FAILED, "", false);
application.handle(event);
rmDispatcher.await();
assertAppState(RMAppState.ACCEPTED, application);
@@ -680,7 +680,7 @@ public class TestRMAppTransitions {
// after max application attempts
RMAppEvent event =
new RMAppFailedAttemptEvent(application.getApplicationId(),
- RMAppEventType.ATTEMPT_FAILED, "");
+ RMAppEventType.ATTEMPT_FAILED, "", false);
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
@@ -804,7 +804,7 @@ public class TestRMAppTransitions {
// KILLED => KILLED event RMAppEventType.ATTEMPT_FAILED
event =
new RMAppFailedAttemptEvent(application.getApplicationId(),
- RMAppEventType.ATTEMPT_FAILED, "");
+ RMAppEventType.ATTEMPT_FAILED, "", false);
application.handle(event);
rmDispatcher.await();
assertTimesAtFinish(application);
Modified: hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java?rev=1558303&r1=1558302&r2=1558303&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java (original)
+++ hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java Wed Jan 15 06:06:31 2014
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.record
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -68,10 +69,10 @@ import org.apache.hadoop.yarn.server.res
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
@@ -120,14 +121,15 @@ public class TestRMAppAttemptTransitions
private AMLivelinessMonitor amFinishingMonitor;
private RMStateStore store;
- private RMApp application;
+ private RMAppImpl application;
private RMAppAttempt applicationAttempt;
private Configuration conf = new Configuration();
private AMRMTokenSecretManager amRMTokenManager = spy(new AMRMTokenSecretManager(conf));
private ClientToAMTokenSecretManagerInRM clientToAMTokenManager =
spy(new ClientToAMTokenSecretManagerInRM());
-
+ private boolean transferStateFromPreviousAttempt = false;
+
private final class TestApplicationAttemptEventDispatcher implements
EventHandler<RMAppAttemptEvent> {
@@ -150,6 +152,11 @@ public class TestRMAppAttemptTransitions
@Override
public void handle(RMAppEvent event) {
assertEquals(application.getApplicationId(), event.getApplicationId());
+ if (event instanceof RMAppFailedAttemptEvent) {
+ transferStateFromPreviousAttempt =
+ ((RMAppFailedAttemptEvent) event)
+ .getTransferStateFromPreviousAttempt();
+ }
try {
application.handle(event);
} catch (Throwable t) {
@@ -254,10 +261,10 @@ public class TestRMAppAttemptTransitions
unmanagedAM = false;
- application = mock(RMApp.class);
+ application = mock(RMAppImpl.class);
applicationAttempt =
new RMAppAttemptImpl(applicationAttemptId, rmContext, scheduler,
- masterService, submissionContext, new Configuration());
+ masterService, submissionContext, new Configuration(), false);
when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt);
when(application.getApplicationId()).thenReturn(applicationId);
@@ -371,6 +378,7 @@ public class TestRMAppAttemptTransitions
assertNull(applicationAttempt.getFinalApplicationStatus());
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
verifyAttemptFinalStateSaved();
+ assertFalse(transferStateFromPreviousAttempt);
}
/**
@@ -525,6 +533,7 @@ public class TestRMAppAttemptTransitions
assertEquals(container, applicationAttempt.getMasterContainer());
assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
+ assertFalse(transferStateFromPreviousAttempt);
}
@@ -654,6 +663,7 @@ public class TestRMAppAttemptTransitions
diagnostics));
testAppAttemptFinishedState(null, finalStatus, url, diagnostics, 1,
true);
+ assertFalse(transferStateFromPreviousAttempt);
}
private void sendAttemptUpdateSavedEvent(RMAppAttempt applicationAttempt) {
@@ -682,6 +692,21 @@ public class TestRMAppAttemptTransitions
}
@Test
+ public void testUnmanagedAMContainersCleanup() {
+ unmanagedAM = true;
+ when(submissionContext.getUnmanagedAM()).thenReturn(true);
+ when(submissionContext.getKeepContainersAcrossApplicationAttempts())
+ .thenReturn(true);
+ // submit AM and check it goes to SUBMITTED state
+ submitApplicationAttempt();
+ // launch AM and verify attempt failed
+ applicationAttempt.handle(new RMAppAttemptRegistrationEvent(
+ applicationAttempt.getAppAttemptId(), "host", 8042, "oldtrackingurl"));
+ sendAttemptUpdateSavedEvent(applicationAttempt);
+ assertFalse(transferStateFromPreviousAttempt);
+ }
+
+ @Test
public void testNewToKilled() {
applicationAttempt.handle(
new RMAppAttemptEvent(
@@ -1092,6 +1117,64 @@ public class TestRMAppAttemptTransitions
Assert.assertNull(token);
}
+ @Test
+ public void testFailedToFailed() {
+ // create a failed attempt.
+ when(submissionContext.getKeepContainersAcrossApplicationAttempts())
+ .thenReturn(true);
+ Container amContainer = allocateApplicationAttempt();
+ launchApplicationAttempt(amContainer);
+ runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
+ ContainerStatus cs1 =
+ ContainerStatus.newInstance(amContainer.getId(),
+ ContainerState.COMPLETE, "some error", 123);
+ ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
+ applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
+ appAttemptId, cs1));
+ sendAttemptUpdateSavedEvent(applicationAttempt);
+ assertEquals(RMAppAttemptState.FAILED,
+ applicationAttempt.getAppAttemptState());
+ // should not kill containers when attempt fails.
+ assertTrue(transferStateFromPreviousAttempt);
+
+ // failed attempt captured the container finished event.
+ assertEquals(0, applicationAttempt.getJustFinishedContainers().size());
+ ContainerStatus cs2 =
+ ContainerStatus.newInstance(ContainerId.newInstance(appAttemptId, 2),
+ ContainerState.COMPLETE, "", 0);
+ applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
+ appAttemptId, cs2));
+ assertEquals(1, applicationAttempt.getJustFinishedContainers().size());
+ assertEquals(cs2.getContainerId(), applicationAttempt
+ .getJustFinishedContainers().get(0).getContainerId());
+ }
+
+
+ @Test
+ public void testContainersCleanupForLastAttempt() {
+ // create a failed attempt.
+ applicationAttempt =
+ new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), rmContext,
+ scheduler, masterService, submissionContext, new Configuration(),
+ true);
+ when(submissionContext.getKeepContainersAcrossApplicationAttempts())
+ .thenReturn(true);
+ when(submissionContext.getMaxAppAttempts()).thenReturn(1);
+ Container amContainer = allocateApplicationAttempt();
+ launchApplicationAttempt(amContainer);
+ runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
+ ContainerStatus cs1 =
+ ContainerStatus.newInstance(amContainer.getId(),
+ ContainerState.COMPLETE, "some error", 123);
+ ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
+ applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
+ appAttemptId, cs1));
+ sendAttemptUpdateSavedEvent(applicationAttempt);
+ assertEquals(RMAppAttemptState.FAILED,
+ applicationAttempt.getAppAttemptState());
+ assertFalse(transferStateFromPreviousAttempt);
+ }
+
private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) {
verify(amRMTokenManager, times(count)).applicationMasterFinished(appAttemptId);
if (UserGroupInformation.isSecurityEnabled()) {
Modified: hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java?rev=1558303&r1=1558302&r2=1558303&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java (original)
+++ hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java Wed Jan 15 06:06:31 2014
@@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.api.record
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.resource.Resources;
@@ -66,8 +66,10 @@ public class TestQueueMetrics {
MetricsSource queueSource= queueSource(ms, queueName);
AppSchedulingInfo app = mockApp(user);
- metrics.submitApp(user, 1);
+ metrics.submitApp(user);
MetricsSource userSource = userSource(ms, queueName, user);
+ checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
+ metrics.submitAppAttempt(user);
checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
metrics.setAvailableResourcesToQueue(Resources.createResource(100*GB, 100));
@@ -76,7 +78,7 @@ public class TestQueueMetrics {
// configurable cluster/queue resources
checkResources(queueSource, 0, 0, 0, 0, 0, 100*GB, 100, 15*GB, 15, 5, 0, 0, 0);
- metrics.incrAppsRunning(app, user);
+ metrics.runAppAttempt(app.getApplicationId(), user);
checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
metrics.allocateResources(user, 3, Resources.createResource(2*GB, 2));
@@ -85,7 +87,10 @@ public class TestQueueMetrics {
metrics.releaseResources(user, 1, Resources.createResource(2*GB, 2));
checkResources(queueSource, 4*GB, 4, 2, 3, 1, 100*GB, 100, 9*GB, 9, 2, 0, 0, 0);
- metrics.finishApp(app, RMAppAttemptState.FINISHED);
+ metrics.finishAppAttempt(
+ app.getApplicationId(), app.isPending(), app.getUser());
+ checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
+ metrics.finishApp(user, RMAppState.FINISHED);
checkApps(queueSource, 1, 0, 0, 1, 0, 0, true);
assertNull(userSource);
}
@@ -100,39 +105,47 @@ public class TestQueueMetrics {
MetricsSource queueSource = queueSource(ms, queueName);
AppSchedulingInfo app = mockApp(user);
- metrics.submitApp(user, 1);
+ metrics.submitApp(user);
MetricsSource userSource = userSource(ms, queueName, user);
+ checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
+ metrics.submitAppAttempt(user);
checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
- metrics.incrAppsRunning(app, user);
+ metrics.runAppAttempt(app.getApplicationId(), user);
checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
- metrics.finishApp(app, RMAppAttemptState.FAILED);
- checkApps(queueSource, 1, 0, 0, 0, 1, 0, true);
+ metrics.finishAppAttempt(
+ app.getApplicationId(), app.isPending(), app.getUser());
+ checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
// As the application has failed, framework retries the same application
// based on configuration
- metrics.submitApp(user, 2);
+ metrics.submitAppAttempt(user);
checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
- metrics.incrAppsRunning(app, user);
+ metrics.runAppAttempt(app.getApplicationId(), user);
checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
// Suppose say application has failed this time as well.
- metrics.finishApp(app, RMAppAttemptState.FAILED);
- checkApps(queueSource, 1, 0, 0, 0, 1, 0, true);
+ metrics.finishAppAttempt(
+ app.getApplicationId(), app.isPending(), app.getUser());
+ checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
// As the application has failed, framework retries the same application
// based on configuration
- metrics.submitApp(user, 3);
+ metrics.submitAppAttempt(user);
checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
- metrics.incrAppsRunning(app, user);
+ metrics.runAppAttempt(app.getApplicationId(), user);
checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
- // Suppose say application has finished.
- metrics.finishApp(app, RMAppAttemptState.FINISHED);
- checkApps(queueSource, 1, 0, 0, 1, 0, 0, true);
+ // Suppose say application has failed, and there's no more retries.
+ metrics.finishAppAttempt(
+ app.getApplicationId(), app.isPending(), app.getUser());
+ checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
+
+ metrics.finishApp(user, RMAppState.FAILED);
+ checkApps(queueSource, 1, 0, 0, 0, 1, 0, true);
assertNull(userSource);
}
@@ -146,9 +159,13 @@ public class TestQueueMetrics {
MetricsSource queueSource = queueSource(ms, queueName);
AppSchedulingInfo app = mockApp(user);
- metrics.submitApp(user, 1);
+ metrics.submitApp(user);
MetricsSource userSource = userSource(ms, queueName, user);
+ checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
+ checkApps(userSource, 1, 0, 0, 0, 0, 0, true);
+
+ metrics.submitAppAttempt(user);
checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
checkApps(userSource, 1, 1, 0, 0, 0, 0, true);
@@ -160,7 +177,7 @@ public class TestQueueMetrics {
checkResources(queueSource, 0, 0, 0, 0, 0, 100*GB, 100, 15*GB, 15, 5, 0, 0, 0);
checkResources(userSource, 0, 0, 0, 0, 0, 10*GB, 10, 15*GB, 15, 5, 0, 0, 0);
- metrics.incrAppsRunning(app, user);
+ metrics.runAppAttempt(app.getApplicationId(), user);
checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
checkApps(userSource, 1, 0, 1, 0, 0, 0, true);
@@ -172,7 +189,11 @@ public class TestQueueMetrics {
checkResources(queueSource, 4*GB, 4, 2, 3, 1, 100*GB, 100, 9*GB, 9, 2, 0, 0, 0);
checkResources(userSource, 4*GB, 4, 2, 3, 1, 10*GB, 10, 9*GB, 9, 2, 0, 0, 0);
- metrics.finishApp(app, RMAppAttemptState.FINISHED);
+ metrics.finishAppAttempt(
+ app.getApplicationId(), app.isPending(), app.getUser());
+ checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
+ checkApps(userSource, 1, 0, 0, 0, 0, 0, true);
+ metrics.finishApp(user, RMAppState.FINISHED);
checkApps(queueSource, 1, 0, 0, 1, 0, 0, true);
checkApps(userSource, 1, 0, 0, 1, 0, 0, true);
}
@@ -192,10 +213,16 @@ public class TestQueueMetrics {
MetricsSource queueSource = queueSource(ms, leafQueueName);
AppSchedulingInfo app = mockApp(user);
- metrics.submitApp(user, 1);
+ metrics.submitApp(user);
MetricsSource userSource = userSource(ms, leafQueueName, user);
MetricsSource parentUserSource = userSource(ms, parentQueueName, user);
+ checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
+ checkApps(parentQueueSource, 1, 0, 0, 0, 0, 0, true);
+ checkApps(userSource, 1, 0, 0, 0, 0, 0, true);
+ checkApps(parentUserSource, 1, 0, 0, 0, 0, 0, true);
+
+ metrics.submitAppAttempt(user);
checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
checkApps(parentQueueSource, 1, 1, 0, 0, 0, 0, true);
checkApps(userSource, 1, 1, 0, 0, 0, 0, true);
@@ -211,7 +238,7 @@ public class TestQueueMetrics {
checkResources(userSource, 0, 0, 0, 0, 0, 10*GB, 10, 15*GB, 15, 5, 0, 0, 0);
checkResources(parentUserSource, 0, 0, 0, 0, 0, 10*GB, 10, 15*GB, 15, 5, 0, 0, 0);
- metrics.incrAppsRunning(app, user);
+ metrics.runAppAttempt(app.getApplicationId(), user);
checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
checkApps(userSource, 1, 0, 1, 0, 0, 0, true);
@@ -231,7 +258,14 @@ public class TestQueueMetrics {
checkResources(userSource, 4*GB, 4, 2, 3, 1, 10*GB, 10, 9*GB, 9, 2, 0, 0, 0);
checkResources(parentUserSource, 4*GB, 4, 2, 3, 1, 10*GB, 10, 9*GB, 9, 2, 0, 0, 0);
- metrics.finishApp(app, RMAppAttemptState.FINISHED);
+ metrics.finishAppAttempt(
+ app.getApplicationId(), app.isPending(), app.getUser());
+ checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
+ checkApps(parentQueueSource, 1, 0, 0, 0, 0, 0, true);
+ checkApps(userSource, 1, 0, 0, 0, 0, 0, true);
+ checkApps(parentUserSource, 1, 0, 0, 0, 0, 0, true);
+
+ metrics.finishApp(user, RMAppState.FINISHED);
checkApps(queueSource, 1, 0, 0, 1, 0, 0, true);
checkApps(parentQueueSource, 1, 0, 0, 1, 0, 0, true);
checkApps(userSource, 1, 0, 0, 1, 0, 0, true);
@@ -308,7 +342,7 @@ public class TestQueueMetrics {
assertGauge("AppsPending", pending, rb);
assertGauge("AppsRunning", running, rb);
assertCounter("AppsCompleted", completed, rb);
- assertGauge("AppsFailed", failed, rb);
+ assertCounter("AppsFailed", failed, rb);
assertCounter("AppsKilled", killed, rb);
}
Modified: hadoop/common/branches/HDFS-5535/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
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-5535/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?rev=1558303&r1=1558302&r2=1558303&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-5535/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 (original)
+++ hadoop/common/branches/HDFS-5535/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 Wed Jan 15 06:06:31 2014
@@ -562,18 +562,18 @@ public class TestCapacityScheduler {
new AppAddedSchedulerEvent(appId, "default", "user");
cs.handle(addAppEvent);
SchedulerEvent addAttemptEvent =
- new AppAttemptAddedSchedulerEvent(appAttemptId);
+ new AppAttemptAddedSchedulerEvent(appAttemptId, false);
cs.handle(addAttemptEvent);
// Verify the blacklist can be updated independent of requesting containers
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(),
Collections.singletonList(host), null);
- Assert.assertTrue(cs.getApplication(appAttemptId).isBlacklisted(host));
+ Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(), null,
Collections.singletonList(host));
- Assert.assertFalse(cs.getApplication(appAttemptId).isBlacklisted(host));
+ Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
rm.stop();
}
@@ -597,66 +597,6 @@ public class TestCapacityScheduler {
assertTrue(appComparator.compare(app1, app3) < 0);
assertTrue(appComparator.compare(app2, app3) < 0);
}
-
- @Test
- public void testConcurrentAccessOnApplications() throws Exception {
- CapacityScheduler cs = new CapacityScheduler();
- verifyConcurrentAccessOnApplications(
- cs.appAttempts, FiCaSchedulerApp.class, Queue.class);
- }
-
- public static <T extends SchedulerApplicationAttempt, Q extends Queue>
- void verifyConcurrentAccessOnApplications(
- final Map<ApplicationAttemptId, T> applications, Class<T> appClazz,
- final Class<Q> queueClazz)
- throws Exception {
- final int size = 10000;
- final ApplicationId appId = ApplicationId.newInstance(0, 0);
- final Constructor<T> ctor = appClazz.getDeclaredConstructor(
- ApplicationAttemptId.class, String.class, queueClazz,
- ActiveUsersManager.class, RMContext.class);
-
- ApplicationAttemptId appAttemptId0
- = ApplicationAttemptId.newInstance(appId, 0);
- applications.put(appAttemptId0, ctor.newInstance(
- appAttemptId0, null, mock(queueClazz), null, null));
- assertNotNull(applications.get(appAttemptId0));
-
- // Imitating the thread of scheduler that will add and remove apps
- final AtomicBoolean finished = new AtomicBoolean(false);
- final AtomicBoolean failed = new AtomicBoolean(false);
- Thread t = new Thread() {
-
- @Override
- public void run() {
- for (int i = 1; i <= size; ++i) {
- ApplicationAttemptId appAttemptId
- = ApplicationAttemptId.newInstance(appId, i);
- try {
- applications.put(appAttemptId, ctor.newInstance(
- appAttemptId, null, mock(queueClazz), null, null));
- } catch (Exception e) {
- failed.set(true);
- finished.set(true);
- return;
- }
- }
- for (int i = 1; i <= size; ++i) {
- ApplicationAttemptId appAttemptId
- = ApplicationAttemptId.newInstance(appId, i);
- applications.remove(appAttemptId);
- }
- finished.set(true);
- }
- };
- t.start();
-
- // Imitating the thread of rmappattempt that will get the app
- while (!finished.get()) {
- assertNotNull(applications.get(appAttemptId0));
- }
- assertFalse(failed.get());
- }
@Test
public void testGetAppsInQueue() throws Exception {
@@ -702,7 +642,7 @@ public class TestCapacityScheduler {
SchedulerApplication app =
TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler(
- cs.applications, cs, "a1");
+ cs.getSchedulerApplications(), cs, "a1");
Assert.assertEquals("a1", app.getQueue().getQueueName());
}
}
Modified: hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java?rev=1558303&r1=1558302&r2=1558303&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java (original)
+++ hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java Wed Jan 15 06:06:31 2014
@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.conf.YarnC
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@@ -63,7 +64,10 @@ import org.apache.hadoop.yarn.server.res
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -345,15 +349,20 @@ public class TestLeafQueue {
.getMockApplicationAttemptId(0, 1);
FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a, null,
rmContext);
- a.submitApplicationAttempt(app_0, user_0);
-
- when(cs.getApplication(appAttemptId_0)).thenReturn(app_0);
+ AppAddedSchedulerEvent addAppEvent =
+ new AppAddedSchedulerEvent(appAttemptId_0.getApplicationId(),
+ a.getQueueName(), user_0);
+ cs.handle(addAppEvent);
+ AppAttemptAddedSchedulerEvent addAttemptEvent =
+ new AppAttemptAddedSchedulerEvent(appAttemptId_0, false);
+ cs.handle(addAttemptEvent);
+
AppAttemptRemovedSchedulerEvent event = new AppAttemptRemovedSchedulerEvent(
- appAttemptId_0, RMAppAttemptState.FAILED);
+ appAttemptId_0, RMAppAttemptState.FAILED, false);
cs.handle(event);
assertEquals(0, a.getMetrics().getAppsPending());
- assertEquals(1, a.getMetrics().getAppsFailed());
+ assertEquals(0, a.getMetrics().getAppsFailed());
// Attempt the same application again
final ApplicationAttemptId appAttemptId_1 = TestUtils
@@ -365,10 +374,12 @@ public class TestLeafQueue {
assertEquals(1, a.getMetrics().getAppsSubmitted());
assertEquals(1, a.getMetrics().getAppsPending());
- when(cs.getApplication(appAttemptId_1)).thenReturn(app_0);
event = new AppAttemptRemovedSchedulerEvent(appAttemptId_0,
- RMAppAttemptState.FINISHED);
+ RMAppAttemptState.FINISHED, false);
cs.handle(event);
+ AppRemovedSchedulerEvent rEvent = new AppRemovedSchedulerEvent(
+ appAttemptId_0.getApplicationId(), RMAppState.FINISHED);
+ cs.handle(rEvent);
assertEquals(1, a.getMetrics().getAppsSubmitted());
assertEquals(0, a.getMetrics().getAppsPending());
Modified: hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java?rev=1558303&r1=1558302&r2=1558303&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java (original)
+++ hadoop/common/branches/HDFS-5535/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java Wed Jan 15 06:06:31 2014
@@ -79,7 +79,6 @@ import org.apache.hadoop.yarn.server.res
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
@@ -260,8 +259,8 @@ public class TestFairScheduler {
scheduler.addApplication(id.getApplicationId(), queueId, userId);
// This conditional is for testAclSubmitApplication where app is rejected
// and no app is added.
- if (scheduler.applications.containsKey(id.getApplicationId())) {
- scheduler.addApplicationAttempt(id);
+ if (scheduler.getSchedulerApplications().containsKey(id.getApplicationId())) {
+ scheduler.addApplicationAttempt(id, false);
}
List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
ResourceRequest request = createResourceRequest(memory, vcores, ResourceRequest.ANY,
@@ -590,7 +589,7 @@ public class TestFairScheduler {
// Make sure queue 2 is waiting with a reservation
assertEquals(0, scheduler.getQueueManager().getQueue("queue2").
getResourceUsage().getMemory());
- assertEquals(1024, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory());
+ assertEquals(1024, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
// Now another node checks in with capacity
RMNode node2 =
@@ -606,10 +605,10 @@ public class TestFairScheduler {
getResourceUsage().getMemory());
// The old reservation should still be there...
- assertEquals(1024, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory());
+ assertEquals(1024, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
// ... but it should disappear when we update the first node.
scheduler.handle(updateEvent);
- assertEquals(0, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory());
+ assertEquals(0, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
}
@@ -630,7 +629,7 @@ public class TestFairScheduler {
"user1");
scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attempAddedEvent =
- new AppAttemptAddedSchedulerEvent(appAttemptId);
+ new AppAttemptAddedSchedulerEvent(appAttemptId, false);
scheduler.handle(attempAddedEvent);
assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
.getRunnableAppSchedulables().size());
@@ -656,7 +655,7 @@ public class TestFairScheduler {
"user2");
scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attempAddedEvent =
- new AppAttemptAddedSchedulerEvent(appAttemptId);
+ new AppAttemptAddedSchedulerEvent(appAttemptId, false);
scheduler.handle(attempAddedEvent);
assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true)
.getRunnableAppSchedulables().size());
@@ -710,7 +709,6 @@ public class TestFairScheduler {
scheduler.reinitialize(conf, resourceManager.getRMContext());
ApplicationAttemptId appId;
- Map<ApplicationAttemptId, FSSchedulerApp> apps = scheduler.appAttempts;
List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
rules.add(new QueuePlacementRule.Specified().initialize(true, null));
@@ -723,17 +721,17 @@ public class TestFairScheduler {
scheduler.getAllocationConfiguration().placementPolicy =
new QueuePlacementPolicy(rules, queues, conf);
appId = createSchedulingRequest(1024, "somequeue", "user1");
- assertEquals("root.somequeue", apps.get(appId).getQueueName());
+ assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user1");
- assertEquals("root.user1", apps.get(appId).getQueueName());
+ assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user3");
- assertEquals("root.user3group", apps.get(appId).getQueueName());
+ assertEquals("root.user3group", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user4");
- assertEquals("root.user4subgroup1", apps.get(appId).getQueueName());
+ assertEquals("root.user4subgroup1", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user5");
- assertEquals("root.user5subgroup2", apps.get(appId).getQueueName());
+ assertEquals("root.user5subgroup2", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "otheruser");
- assertEquals("root.default", apps.get(appId).getQueueName());
+ assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName());
// test without specified as first rule
rules = new ArrayList<QueuePlacementRule>();
@@ -743,11 +741,11 @@ public class TestFairScheduler {
scheduler.getAllocationConfiguration().placementPolicy =
new QueuePlacementPolicy(rules, queues, conf);
appId = createSchedulingRequest(1024, "somequeue", "user1");
- assertEquals("root.user1", apps.get(appId).getQueueName());
+ assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "somequeue", "otheruser");
- assertEquals("root.somequeue", apps.get(appId).getQueueName());
+ assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "otheruser");
- assertEquals("root.default", apps.get(appId).getQueueName());
+ assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName());
}
@Test
@@ -802,13 +800,13 @@ public class TestFairScheduler {
ApplicationAttemptId id11 = createAppAttemptId(1, 1);
scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1");
- scheduler.addApplicationAttempt(id11);
+ scheduler.addApplicationAttempt(id11, false);
ApplicationAttemptId id21 = createAppAttemptId(2, 1);
scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1");
- scheduler.addApplicationAttempt(id21);
+ scheduler.addApplicationAttempt(id21, false);
ApplicationAttemptId id22 = createAppAttemptId(2, 2);
scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1");
- scheduler.addApplicationAttempt(id22);
+ scheduler.addApplicationAttempt(id22, false);
int minReqSize =
FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
@@ -854,7 +852,7 @@ public class TestFairScheduler {
"user1");
scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attemptAddedEvent =
- new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1));
+ new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false);
scheduler.handle(attemptAddedEvent);
// Scheduler should have two queues (the default and the one created for user1)
@@ -865,7 +863,7 @@ public class TestFairScheduler {
.getRunnableAppSchedulables().size());
AppAttemptRemovedSchedulerEvent appRemovedEvent1 = new AppAttemptRemovedSchedulerEvent(
- createAppAttemptId(1, 1), RMAppAttemptState.FINISHED);
+ createAppAttemptId(1, 1), RMAppAttemptState.FINISHED, false);
// Now remove app
scheduler.handle(appRemovedEvent1);
@@ -1138,12 +1136,12 @@ public class TestFairScheduler {
scheduler.handle(nodeUpdate3);
}
- assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size());
- assertEquals(1, scheduler.appAttempts.get(app2).getLiveContainers().size());
- assertEquals(1, scheduler.appAttempts.get(app3).getLiveContainers().size());
- assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
- assertEquals(1, scheduler.appAttempts.get(app5).getLiveContainers().size());
- assertEquals(1, scheduler.appAttempts.get(app6).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app2).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app3).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app5).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app6).getLiveContainers().size());
// Now new requests arrive from queues C and D
ApplicationAttemptId app7 =
@@ -1166,16 +1164,16 @@ public class TestFairScheduler {
// Make sure it is lowest priority container.
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
Resources.createResource(2 * 1024));
- assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size());
- assertEquals(1, scheduler.appAttempts.get(app2).getLiveContainers().size());
- assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
- assertEquals(1, scheduler.appAttempts.get(app5).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app2).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app5).getLiveContainers().size());
// First verify we are adding containers to preemption list for the application
- assertTrue(!Collections.disjoint(scheduler.appAttempts.get(app3).getLiveContainers(),
- scheduler.appAttempts.get(app3).getPreemptionContainers()));
- assertTrue(!Collections.disjoint(scheduler.appAttempts.get(app6).getLiveContainers(),
- scheduler.appAttempts.get(app6).getPreemptionContainers()));
+ assertTrue(!Collections.disjoint(scheduler.getSchedulerApp(app3).getLiveContainers(),
+ scheduler.getSchedulerApp(app3).getPreemptionContainers()));
+ assertTrue(!Collections.disjoint(scheduler.getSchedulerApp(app6).getLiveContainers(),
+ scheduler.getSchedulerApp(app6).getPreemptionContainers()));
// Pretend 15 seconds have passed
clock.tick(15);
@@ -1185,8 +1183,8 @@ public class TestFairScheduler {
Resources.createResource(2 * 1024));
// At this point the containers should have been killed (since we are not simulating AM)
- assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size());
- assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
// Trigger a kill by insisting we want containers back
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
@@ -1200,22 +1198,22 @@ public class TestFairScheduler {
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
Resources.createResource(2 * 1024));
- assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size());
- assertEquals(0, scheduler.appAttempts.get(app2).getLiveContainers().size());
- assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size());
- assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
- assertEquals(0, scheduler.appAttempts.get(app5).getLiveContainers().size());
- assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(app2).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(app5).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
// Now A and B are below fair share, so preemption shouldn't do anything
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
Resources.createResource(2 * 1024));
- assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size());
- assertEquals(0, scheduler.appAttempts.get(app2).getLiveContainers().size());
- assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size());
- assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
- assertEquals(0, scheduler.appAttempts.get(app5).getLiveContainers().size());
- assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(app2).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(app5).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
}
@Test (timeout = 5000)
@@ -1374,9 +1372,9 @@ public class TestFairScheduler {
// One container should get reservation and the other should get nothing
assertEquals(1024,
- scheduler.appAttempts.get(attId1).getCurrentReservation().getMemory());
+ scheduler.getSchedulerApp(attId1).getCurrentReservation().getMemory());
assertEquals(0,
- scheduler.appAttempts.get(attId2).getCurrentReservation().getMemory());
+ scheduler.getSchedulerApp(attId2).getCurrentReservation().getMemory());
}
@Test (timeout = 5000)
@@ -1411,7 +1409,7 @@ public class TestFairScheduler {
scheduler.handle(updateEvent);
// App 1 should be running
- assertEquals(1, scheduler.appAttempts.get(attId1).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(attId1).getLiveContainers().size());
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
"user1", 1);
@@ -1420,7 +1418,7 @@ public class TestFairScheduler {
scheduler.handle(updateEvent);
// App 2 should not be running
- assertEquals(0, scheduler.appAttempts.get(attId2).getLiveContainers().size());
+ assertEquals(0, scheduler.getSchedulerApp(attId2).getLiveContainers().size());
// Request another container for app 1
createSchedulingRequestExistingApplication(1024, 1, attId1);
@@ -1429,7 +1427,7 @@ public class TestFairScheduler {
scheduler.handle(updateEvent);
// Request should be fulfilled
- assertEquals(2, scheduler.appAttempts.get(attId1).getLiveContainers().size());
+ assertEquals(2, scheduler.getSchedulerApp(attId1).getLiveContainers().size());
}
@Test (timeout = 5000)
@@ -1449,10 +1447,10 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
scheduler.handle(updateEvent);
- FSSchedulerApp app = scheduler.appAttempts.get(attId);
+ FSSchedulerApp app = scheduler.getSchedulerApp(attId);
assertEquals(1, app.getLiveContainers().size());
- ContainerId containerId = scheduler.appAttempts.get(attId)
+ ContainerId containerId = scheduler.getSchedulerApp(attId)
.getLiveContainers().iterator().next().getContainerId();
// Cause reservation to be created
@@ -1521,9 +1519,9 @@ public class TestFairScheduler {
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
"norealuserhasthisname2", 1);
- FSSchedulerApp app1 = scheduler.appAttempts.get(attId1);
+ FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
assertNotNull("The application was not allowed", app1);
- FSSchedulerApp app2 = scheduler.appAttempts.get(attId2);
+ FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
assertNull("The application was allowed", app2);
}
@@ -1547,7 +1545,7 @@ public class TestFairScheduler {
ApplicationAttemptId appId = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
scheduler.addApplication(appId.getApplicationId(), "queue1", "user1");
- scheduler.addApplicationAttempt(appId);
+ scheduler.addApplicationAttempt(appId, false);
// 1 request with 2 nodes on the same rack. another request with 1 node on
// a different rack
@@ -1566,14 +1564,14 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1);
scheduler.handle(updateEvent1);
// should assign node local
- assertEquals(1, scheduler.appAttempts.get(appId).getLiveContainers().size());
+ assertEquals(1, scheduler.getSchedulerApp(appId).getLiveContainers().size());
// node 2 checks in
scheduler.update();
NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
scheduler.handle(updateEvent2);
// should assign rack local
- assertEquals(2, scheduler.appAttempts.get(appId).getLiveContainers().size());
+ assertEquals(2, scheduler.getSchedulerApp(appId).getLiveContainers().size());
}
@Test (timeout = 5000)
@@ -1592,8 +1590,8 @@ public class TestFairScheduler {
"user1", 2);
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
"user1", 2);
- FSSchedulerApp app1 = scheduler.appAttempts.get(attId1);
- FSSchedulerApp app2 = scheduler.appAttempts.get(attId2);
+ FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
+ FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true);
queue1.setPolicy(new FifoPolicy());
@@ -1633,7 +1631,7 @@ public class TestFairScheduler {
ApplicationAttemptId attId =
createSchedulingRequest(1024, "root.default", "user", 8);
- FSSchedulerApp app = scheduler.appAttempts.get(attId);
+ FSSchedulerApp app = scheduler.getSchedulerApp(attId);
// set maxAssign to 2: only 2 containers should be allocated
scheduler.maxAssign = 2;
@@ -1695,10 +1693,10 @@ public class TestFairScheduler {
ApplicationAttemptId attId4 =
createSchedulingRequest(1024, fifoQueue, user, 4);
- FSSchedulerApp app1 = scheduler.appAttempts.get(attId1);
- FSSchedulerApp app2 = scheduler.appAttempts.get(attId2);
- FSSchedulerApp app3 = scheduler.appAttempts.get(attId3);
- FSSchedulerApp app4 = scheduler.appAttempts.get(attId4);
+ FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
+ FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
+ FSSchedulerApp app3 = scheduler.getSchedulerApp(attId3);
+ FSSchedulerApp app4 = scheduler.getSchedulerApp(attId4);
scheduler.getQueueManager().getLeafQueue(fifoQueue, true)
.setPolicy(SchedulingPolicy.parse("fifo"));
@@ -1813,7 +1811,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
scheduler.handle(updateEvent);
- FSSchedulerApp app = scheduler.appAttempts.get(attId);
+ FSSchedulerApp app = scheduler.getSchedulerApp(attId);
assertEquals(0, app.getLiveContainers().size());
assertEquals(0, app.getReservedContainers().size());
@@ -1882,7 +1880,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2);
// no matter how many heartbeats, node2 should never get a container
- FSSchedulerApp app = scheduler.appAttempts.get(attId1);
+ FSSchedulerApp app = scheduler.getSchedulerApp(attId1);
for (int i = 0; i < 10; i++) {
scheduler.handle(node2UpdateEvent);
assertEquals(0, app.getLiveContainers().size());
@@ -1921,7 +1919,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2);
// no matter how many heartbeats, node2 should never get a container
- FSSchedulerApp app = scheduler.appAttempts.get(attId1);
+ FSSchedulerApp app = scheduler.getSchedulerApp(attId1);
for (int i = 0; i < 10; i++) {
scheduler.handle(node2UpdateEvent);
assertEquals(0, app.getLiveContainers().size());
@@ -1954,7 +1952,7 @@ public class TestFairScheduler {
ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1",
"user1", 0);
- FSSchedulerApp app = scheduler.appAttempts.get(attId);
+ FSSchedulerApp app = scheduler.getSchedulerApp(attId);
ResourceRequest nodeRequest = createResourceRequest(1024, node2.getHostName(), 1, 2, true);
ResourceRequest rackRequest = createResourceRequest(1024, "rack1", 1, 2, true);
@@ -1994,7 +1992,7 @@ public class TestFairScheduler {
ApplicationAttemptId attId = createSchedulingRequest(1024, 1, "default",
"user1", 2);
- FSSchedulerApp app = scheduler.appAttempts.get(attId);
+ FSSchedulerApp app = scheduler.getSchedulerApp(attId);
scheduler.update();
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
@@ -2014,10 +2012,10 @@ public class TestFairScheduler {
ApplicationAttemptId appAttId1 = createSchedulingRequest(2048, 1, "queue1",
"user1", 2);
- FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1);
+ FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 2, "queue1",
"user1", 2);
- FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2);
+ FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
drfPolicy.initialize(scheduler.getClusterCapacity());
@@ -2055,13 +2053,13 @@ public class TestFairScheduler {
ApplicationAttemptId appAttId1 = createSchedulingRequest(3072, 1, "queue1",
"user1", 2);
- FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1);
+ FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
ApplicationAttemptId appAttId2 = createSchedulingRequest(2048, 2, "queue1",
"user1", 2);
- FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2);
+ FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
ApplicationAttemptId appAttId3 = createSchedulingRequest(1024, 2, "queue2",
"user1", 2);
- FSSchedulerApp app3 = scheduler.appAttempts.get(appAttId3);
+ FSSchedulerApp app3 = scheduler.getSchedulerApp(appAttId3);
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
drfPolicy.initialize(scheduler.getClusterCapacity());
@@ -2092,19 +2090,19 @@ public class TestFairScheduler {
ApplicationAttemptId appAttId1 = createSchedulingRequest(3074, 1, "queue1.subqueue1",
"user1", 2);
Thread.sleep(3); // so that start times will be different
- FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1);
+ FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 3, "queue1.subqueue1",
"user1", 2);
Thread.sleep(3); // so that start times will be different
- FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2);
+ FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
ApplicationAttemptId appAttId3 = createSchedulingRequest(2048, 2, "queue1.subqueue2",
"user1", 2);
Thread.sleep(3); // so that start times will be different
- FSSchedulerApp app3 = scheduler.appAttempts.get(appAttId3);
+ FSSchedulerApp app3 = scheduler.getSchedulerApp(appAttId3);
ApplicationAttemptId appAttId4 = createSchedulingRequest(1024, 2, "queue2",
"user1", 2);
Thread.sleep(3); // so that start times will be different
- FSSchedulerApp app4 = scheduler.appAttempts.get(appAttId4);
+ FSSchedulerApp app4 = scheduler.getSchedulerApp(appAttId4);
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
drfPolicy.initialize(scheduler.getClusterCapacity());
@@ -2184,7 +2182,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent(node2);
// no matter how many heartbeats, node2 should never get a container
- FSSchedulerApp app = scheduler.appAttempts.get(attId1);
+ FSSchedulerApp app = scheduler.getSchedulerApp(attId1);
for (int i = 0; i < 10; i++) {
scheduler.handle(node2UpdateEvent);
assertEquals(0, app.getLiveContainers().size());
@@ -2195,16 +2193,8 @@ public class TestFairScheduler {
assertEquals(1, app.getLiveContainers().size());
}
- @Test
- public void testConcurrentAccessOnApplications() throws Exception {
- FairScheduler fs = new FairScheduler();
- TestCapacityScheduler.verifyConcurrentAccessOnApplications(
- fs.appAttempts, FSSchedulerApp.class, FSLeafQueue.class);
- }
-
-
private void verifyAppRunnable(ApplicationAttemptId attId, boolean runnable) {
- FSSchedulerApp app = scheduler.appAttempts.get(attId);
+ FSSchedulerApp app = scheduler.getSchedulerApp(attId);
FSLeafQueue queue = app.getQueue();
Collection<AppSchedulable> runnableApps =
queue.getRunnableAppSchedulables();
@@ -2260,7 +2250,7 @@ public class TestFairScheduler {
// Remove app 1 and both app 2 and app 4 should becomes runnable in its place
AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
- new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED);
+ new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED, false);
scheduler.handle(appRemovedEvent1);
verifyAppRunnable(attId2, true);
verifyQueueNumRunnable("queue2", 1, 0);
@@ -2324,7 +2314,7 @@ public class TestFairScheduler {
// Even though the app was removed from sub3, the app from sub2 gets to go
// because it came in first
AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
- new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED);
+ new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED, false);
scheduler.handle(appRemovedEvent1);
verifyAppRunnable(attId4, true);
verifyQueueNumRunnable("queue1.sub2", 2, 0);
@@ -2333,7 +2323,7 @@ public class TestFairScheduler {
// Now test removal of a non-runnable app
AppAttemptRemovedSchedulerEvent appRemovedEvent2 =
- new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.KILLED);
+ new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.KILLED, true);
scheduler.handle(appRemovedEvent2);
assertEquals(0, scheduler.maxRunningEnforcer.usersNonRunnableApps
.get("user1").size());
@@ -2341,7 +2331,7 @@ public class TestFairScheduler {
verifyQueueNumRunnable("queue1.sub3", 0, 0);
// verify it doesn't become runnable when there would be space for it
AppAttemptRemovedSchedulerEvent appRemovedEvent3 =
- new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED);
+ new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED, true);
scheduler.handle(appRemovedEvent3);
verifyQueueNumRunnable("queue1.sub2", 1, 0);
verifyQueueNumRunnable("queue1.sub3", 0, 0);
@@ -2378,7 +2368,7 @@ public class TestFairScheduler {
ApplicationAttemptId appAttemptId =
createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
fs.addApplication(appAttemptId.getApplicationId(), "queue11", "user11");
- fs.addApplicationAttempt(appAttemptId);
+ fs.addApplicationAttempt(appAttemptId, false);
List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
ResourceRequest request =
createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
@@ -2389,7 +2379,7 @@ public class TestFairScheduler {
// at least one pass
Thread.sleep(fs.getConf().getContinuousSchedulingSleepMs() + 500);
- FSSchedulerApp app = fs.appAttempts.get(appAttemptId);
+ FSSchedulerApp app = fs.getSchedulerApp(appAttemptId);
// Wait until app gets resources.
while (app.getCurrentConsumption().equals(Resources.none())) { }
@@ -2477,7 +2467,7 @@ public class TestFairScheduler {
ApplicationAttemptId appAttemptId =
createSchedulingRequest(GB, "root.default", "user", 1);
- FSSchedulerApp app = scheduler.appAttempts.get(appAttemptId);
+ FSSchedulerApp app = scheduler.getSchedulerApp(appAttemptId);
// Verify the blacklist can be updated independent of requesting containers
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
@@ -2487,7 +2477,7 @@ public class TestFairScheduler {
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(), null,
Collections.singletonList(host));
- assertFalse(scheduler.appAttempts.get(appAttemptId).isBlacklisted(host));
+ assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host));
List<ResourceRequest> update = Arrays.asList(
createResourceRequest(GB, node.getHostName(), 1, 0, true));
@@ -2555,6 +2545,6 @@ public class TestFairScheduler {
FairScheduler scheduler =
(FairScheduler) resourceManager.getResourceScheduler();
TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler(
- scheduler.applications, scheduler, "default");
+ scheduler.getSchedulerApplications(), scheduler, "default");
}
}