|
@@ -25,15 +25,29 @@ import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.net.InetSocketAddress;
|
|
|
+import java.security.PrivilegedAction;
|
|
|
import java.util.Collections;
|
|
|
import java.util.Comparator;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.concurrent.BrokenBarrierException;
|
|
|
+import java.util.concurrent.CyclicBarrier;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.net.NetworkTopology;
|
|
|
+import org.apache.hadoop.security.Credentials;
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.security.token.Token;
|
|
|
+import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.LocalConfigurationProvider;
|
|
|
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
@@ -46,13 +60,20 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
+import org.apache.hadoop.yarn.ipc.YarnRPC;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.Application;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
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.Task;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
+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.rmnode.RMNode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
@@ -686,4 +707,125 @@ public class TestCapacityScheduler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 30000)
|
|
|
+ public void testAllocateDoesNotBlockOnSchedulerLock() throws Exception {
|
|
|
+ final YarnConfiguration conf = new YarnConfiguration();
|
|
|
+ conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
|
|
|
+ ResourceScheduler.class);
|
|
|
+ MyContainerManager containerManager = new MyContainerManager();
|
|
|
+ final MockRMWithAMS rm =
|
|
|
+ new MockRMWithAMS(conf, containerManager);
|
|
|
+ rm.start();
|
|
|
+
|
|
|
+ MockNM nm1 = rm.registerNode("localhost:1234", 5120);
|
|
|
+
|
|
|
+ Map<ApplicationAccessType, String> acls =
|
|
|
+ new HashMap<ApplicationAccessType, String>(2);
|
|
|
+ acls.put(ApplicationAccessType.VIEW_APP, "*");
|
|
|
+ RMApp app = rm.submitApp(1024, "appname", "appuser", acls);
|
|
|
+
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
+
|
|
|
+ RMAppAttempt attempt = app.getCurrentAppAttempt();
|
|
|
+ ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
|
|
|
+ int msecToWait = 10000;
|
|
|
+ int msecToSleep = 100;
|
|
|
+ while (attempt.getAppAttemptState() != RMAppAttemptState.LAUNCHED
|
|
|
+ && msecToWait > 0) {
|
|
|
+ LOG.info("Waiting for AppAttempt to reach LAUNCHED state. "
|
|
|
+ + "Current state is " + attempt.getAppAttemptState());
|
|
|
+ Thread.sleep(msecToSleep);
|
|
|
+ msecToWait -= msecToSleep;
|
|
|
+ }
|
|
|
+ Assert.assertEquals(attempt.getAppAttemptState(),
|
|
|
+ RMAppAttemptState.LAUNCHED);
|
|
|
+
|
|
|
+ // Create a client to the RM.
|
|
|
+ final YarnRPC rpc = YarnRPC.create(conf);
|
|
|
+
|
|
|
+ UserGroupInformation currentUser =
|
|
|
+ UserGroupInformation.createRemoteUser(applicationAttemptId.toString());
|
|
|
+ Credentials credentials = containerManager.getContainerCredentials();
|
|
|
+ final InetSocketAddress rmBindAddress =
|
|
|
+ rm.getApplicationMasterService().getBindAddress();
|
|
|
+ Token<? extends TokenIdentifier> amRMToken =
|
|
|
+ MockRMWithAMS.setupAndReturnAMRMToken(rmBindAddress,
|
|
|
+ credentials.getAllTokens());
|
|
|
+ currentUser.addToken(amRMToken);
|
|
|
+ ApplicationMasterProtocol client =
|
|
|
+ currentUser.doAs(new PrivilegedAction<ApplicationMasterProtocol>() {
|
|
|
+ @Override
|
|
|
+ public ApplicationMasterProtocol run() {
|
|
|
+ return (ApplicationMasterProtocol) rpc.getProxy(
|
|
|
+ ApplicationMasterProtocol.class, rmBindAddress, conf);
|
|
|
+ }
|
|
|
+ });
|
|
|
+
|
|
|
+ RegisterApplicationMasterRequest request =
|
|
|
+ RegisterApplicationMasterRequest.newInstance("localhost", 12345, "");
|
|
|
+ client.registerApplicationMaster(request);
|
|
|
+
|
|
|
+ // grab the scheduler lock from another thread
|
|
|
+ // and verify an allocate call in this thread doesn't block on it
|
|
|
+ final CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
|
|
|
+ final CyclicBarrier barrier = new CyclicBarrier(2);
|
|
|
+ Thread otherThread = new Thread(new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ synchronized(cs) {
|
|
|
+ try {
|
|
|
+ barrier.await();
|
|
|
+ barrier.await();
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ e.printStackTrace();
|
|
|
+ } catch (BrokenBarrierException e) {
|
|
|
+ e.printStackTrace();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ });
|
|
|
+ otherThread.start();
|
|
|
+ barrier.await();
|
|
|
+ AllocateRequest allocateRequest =
|
|
|
+ AllocateRequest.newInstance(0, 0.0f, null, null, null);
|
|
|
+ client.allocate(allocateRequest);
|
|
|
+ barrier.await();
|
|
|
+ otherThread.join();
|
|
|
+
|
|
|
+ rm.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testNumClusterNodes() throws Exception {
|
|
|
+ YarnConfiguration conf = new YarnConfiguration();
|
|
|
+ CapacityScheduler cs = new CapacityScheduler();
|
|
|
+ cs.setConf(conf);
|
|
|
+ RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
|
|
|
+ null, new RMContainerTokenSecretManager(conf),
|
|
|
+ new NMTokenSecretManagerInRM(conf),
|
|
|
+ new ClientToAMTokenSecretManagerInRM(), null);
|
|
|
+ cs.setRMContext(rmContext);
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
+ new CapacitySchedulerConfiguration();
|
|
|
+ setupQueueConfiguration(csConf);
|
|
|
+ cs.init(csConf);
|
|
|
+ cs.start();
|
|
|
+ assertEquals(0, cs.getNumClusterNodes());
|
|
|
+
|
|
|
+ RMNode n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1);
|
|
|
+ RMNode n2 = MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 2);
|
|
|
+ cs.handle(new NodeAddedSchedulerEvent(n1));
|
|
|
+ cs.handle(new NodeAddedSchedulerEvent(n2));
|
|
|
+ assertEquals(2, cs.getNumClusterNodes());
|
|
|
+
|
|
|
+ cs.handle(new NodeRemovedSchedulerEvent(n1));
|
|
|
+ assertEquals(1, cs.getNumClusterNodes());
|
|
|
+ cs.handle(new NodeAddedSchedulerEvent(n1));
|
|
|
+ assertEquals(2, cs.getNumClusterNodes());
|
|
|
+ cs.handle(new NodeRemovedSchedulerEvent(n2));
|
|
|
+ cs.handle(new NodeRemovedSchedulerEvent(n1));
|
|
|
+ assertEquals(0, cs.getNumClusterNodes());
|
|
|
+
|
|
|
+ cs.stop();
|
|
|
+ }
|
|
|
}
|