diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index fa1e740..8e5d9dd 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -221,6 +221,14 @@
RM_PREFIX + "nodemanagers.heartbeat-interval-ms";
public static final long DEFAULT_RM_NM_HEARTBEAT_INTERVAL_MS = 1000;
+ /**
+ * Whether ApplicationMasterLauncher use NMClient to start or stop
+ * containers.
+ */
+ public static final String RM_AMLAUNCHER_USING_NMCLIENT =
+ RM_PREFIX + "amlauncher.using.nmclient";
+ public static final boolean DEFAULT_RM_AMLAUNCHER_USING_NMCLIENT = false;
+
//Delegation token related keys
public static final String DELEGATION_KEY_UPDATE_INTERVAL_KEY =
RM_PREFIX + "delegation.key.update-interval";
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
index b2a59b9..cc6a3ce 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
@@ -38,23 +38,5 @@
org.apache.hadoop
hadoop-yarn-common
-
- org.apache.hadoop
- hadoop-yarn-server-resourcemanager
- test
-
-
- org.apache.hadoop
- hadoop-yarn-server-tests
- test
- test-jar
-
-
- org.apache.hadoop
- hadoop-yarn-server-resourcemanager
- test
- test-jar
-
-
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java
deleted file mode 100644
index 8827e08..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.client;
-
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.InetSocketAddress;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.tools.GetGroupsBase;
-import org.apache.hadoop.tools.GetUserMappingsProtocol;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.yarn.api.RMAdminProtocol;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-
-public class GetGroupsForTesting extends GetGroupsBase {
-
- public GetGroupsForTesting(Configuration conf) {
- super(conf);
- }
-
- public GetGroupsForTesting(Configuration conf, PrintStream out) {
- super(conf, out);
- }
-
- @Override
- protected InetSocketAddress getProtocolAddress(Configuration conf)
- throws IOException {
- return conf.getSocketAddr(YarnConfiguration.RM_ADMIN_ADDRESS,
- YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
- YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
- }
-
- @Override
- public void setConf(Configuration conf) {
- conf = new YarnConfiguration(conf);
- super.setConf(conf);
- }
-
- @Override
- protected GetUserMappingsProtocol getUgmProtocol() throws IOException {
- Configuration conf = getConf();
-
- final InetSocketAddress addr = conf.getSocketAddr(
- YarnConfiguration.RM_ADMIN_ADDRESS,
- YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
- YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
- final YarnRPC rpc = YarnRPC.create(conf);
-
- RMAdminProtocol adminProtocol = (RMAdminProtocol) rpc.getProxy(
- RMAdminProtocol.class, addr, getConf());
-
- return adminProtocol;
- }
-
- public static void main(String[] argv) throws Exception {
- int res = ToolRunner.run(new GetGroupsForTesting(new YarnConfiguration()), argv);
- System.exit(res);
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java
deleted file mode 100644
index 6f3bbe1..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java
+++ /dev/null
@@ -1,539 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.client;
-
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.AMRMProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-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;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
-import org.apache.hadoop.yarn.client.AMRMClient.StoredContainerRequest;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.MiniYARNCluster;
-import org.apache.hadoop.yarn.service.Service.STATE;
-import org.apache.hadoop.yarn.util.Records;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-public class TestAMRMClient {
- static Configuration conf = null;
- static MiniYARNCluster yarnCluster = null;
- static YarnClientImpl yarnClient = null;
- static List nodeReports = null;
- static ApplicationAttemptId attemptId = null;
- static int nodeCount = 3;
-
- static Resource capability;
- static Priority priority;
- static String node;
- static String rack;
- static String[] nodes;
- static String[] racks;
-
- @BeforeClass
- public static void setup() throws Exception {
- // start minicluster
- conf = new YarnConfiguration();
- yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
- yarnCluster.init(conf);
- yarnCluster.start();
-
- // start rm client
- yarnClient = new YarnClientImpl();
- yarnClient.init(conf);
- yarnClient.start();
-
- // get node info
- nodeReports = yarnClient.getNodeReports();
-
- priority = Priority.newInstance(1);
- capability = Resource.newInstance(1024, 1);
-
- node = nodeReports.get(0).getNodeId().getHost();
- rack = nodeReports.get(0).getRackName();
- nodes = new String[]{ node };
- racks = new String[]{ rack };
- }
-
- @Before
- public void startApp() throws Exception {
- // submit new app
- GetNewApplicationResponse newApp = yarnClient.getNewApplication();
- ApplicationId appId = newApp.getApplicationId();
-
- ApplicationSubmissionContext appContext = Records
- .newRecord(ApplicationSubmissionContext.class);
- // set the application id
- appContext.setApplicationId(appId);
- // set the application name
- appContext.setApplicationName("Test");
- // Set the priority for the application master
- Priority pri = Records.newRecord(Priority.class);
- pri.setPriority(0);
- appContext.setPriority(pri);
- // Set the queue to which this application is to be submitted in the RM
- appContext.setQueue("default");
- // Set up the container launch context for the application master
- ContainerLaunchContext amContainer = Records
- .newRecord(ContainerLaunchContext.class);
- appContext.setAMContainerSpec(amContainer);
- // unmanaged AM
- appContext.setUnmanagedAM(true);
- // Create the request to send to the applications manager
- SubmitApplicationRequest appRequest = Records
- .newRecord(SubmitApplicationRequest.class);
- appRequest.setApplicationSubmissionContext(appContext);
- // Submit the application to the applications manager
- yarnClient.submitApplication(appContext);
-
- // wait for app to start
- while (true) {
- ApplicationReport appReport = yarnClient.getApplicationReport(appId);
- if (appReport.getYarnApplicationState() == YarnApplicationState.ACCEPTED) {
- attemptId = appReport.getCurrentApplicationAttemptId();
- break;
- }
- }
- }
-
- @After
- public void cancelApp() {
- attemptId = null;
- }
-
- @AfterClass
- public static void tearDown() {
- if (yarnClient != null && yarnClient.getServiceState() == STATE.STARTED) {
- yarnClient.stop();
- }
- if (yarnCluster != null && yarnCluster.getServiceState() == STATE.STARTED) {
- yarnCluster.stop();
- }
- }
-
- @Test (timeout=60000)
- public void testAMRMClientMatchingFit() throws YarnException, IOException {
- AMRMClientImpl amClient = null;
- try {
- // start am rm client
- amClient = new AMRMClientImpl(attemptId);
- amClient.init(conf);
- amClient.start();
- amClient.registerApplicationMaster("Host", 10000, "");
-
- Resource capability1 = Resource.newInstance(1024, 2);
- Resource capability2 = Resource.newInstance(1024, 1);
- Resource capability3 = Resource.newInstance(1000, 2);
- Resource capability4 = Resource.newInstance(2000, 1);
- Resource capability5 = Resource.newInstance(1000, 3);
- Resource capability6 = Resource.newInstance(2000, 1);
-
- StoredContainerRequest storedContainer1 =
- new StoredContainerRequest(capability1, nodes, racks, priority);
- StoredContainerRequest storedContainer2 =
- new StoredContainerRequest(capability2, nodes, racks, priority);
- StoredContainerRequest storedContainer3 =
- new StoredContainerRequest(capability3, nodes, racks, priority);
- StoredContainerRequest storedContainer4 =
- new StoredContainerRequest(capability4, nodes, racks, priority);
- StoredContainerRequest storedContainer5 =
- new StoredContainerRequest(capability5, nodes, racks, priority);
- StoredContainerRequest storedContainer6 =
- new StoredContainerRequest(capability6, nodes, racks, priority);
- amClient.addContainerRequest(storedContainer1);
- amClient.addContainerRequest(storedContainer2);
- amClient.addContainerRequest(storedContainer3);
- amClient.addContainerRequest(storedContainer4);
- amClient.addContainerRequest(storedContainer5);
- amClient.addContainerRequest(storedContainer6);
-
- // test matching of containers
- List extends Collection> matches;
- StoredContainerRequest storedRequest;
- // exact match
- Resource testCapability1 = Resource.newInstance(1024, 2);
- matches = amClient.getMatchingRequests(priority, node, testCapability1);
- verifyMatches(matches, 1);
- storedRequest = matches.get(0).iterator().next();
- assertTrue(storedContainer1 == storedRequest);
- amClient.removeContainerRequest(storedContainer1);
-
- // exact matching with order maintained
- Resource testCapability2 = Resource.newInstance(2000, 1);
- matches = amClient.getMatchingRequests(priority, node, testCapability2);
- verifyMatches(matches, 2);
- // must be returned in the order they were made
- int i = 0;
- for(StoredContainerRequest storedRequest1 : matches.get(0)) {
- if(i++ == 0) {
- assertTrue(storedContainer4 == storedRequest1);
- } else {
- assertTrue(storedContainer6 == storedRequest1);
- }
- }
- amClient.removeContainerRequest(storedContainer6);
-
- // matching with larger container. all requests returned
- Resource testCapability3 = Resource.newInstance(4000, 4);
- matches = amClient.getMatchingRequests(priority, node, testCapability3);
- assert(matches.size() == 4);
-
- Resource testCapability4 = Resource.newInstance(1024, 2);
- matches = amClient.getMatchingRequests(priority, node, testCapability4);
- assert(matches.size() == 2);
- // verify non-fitting containers are not returned and fitting ones are
- for(Collection testSet : matches) {
- assertTrue(testSet.size() == 1);
- StoredContainerRequest testRequest = testSet.iterator().next();
- assertTrue(testRequest != storedContainer4);
- assertTrue(testRequest != storedContainer5);
- assert(testRequest == storedContainer2 ||
- testRequest == storedContainer3);
- }
-
- Resource testCapability5 = Resource.newInstance(512, 4);
- matches = amClient.getMatchingRequests(priority, node, testCapability5);
- assert(matches.size() == 0);
-
- amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
- null, null);
-
- } finally {
- if (amClient != null && amClient.getServiceState() == STATE.STARTED) {
- amClient.stop();
- }
- }
- }
-
- private void verifyMatches(
- List extends Collection> matches,
- int matchSize) {
- assertTrue(matches.size() == 1);
- assertTrue(matches.get(0).size() == matchSize);
- }
-
- @Test (timeout=60000)
- public void testAMRMClientMatchStorage() throws YarnException, IOException {
- AMRMClientImpl amClient = null;
- try {
- // start am rm client
- amClient = new AMRMClientImpl(attemptId);
- amClient.init(conf);
- amClient.start();
- amClient.registerApplicationMaster("Host", 10000, "");
-
- Priority priority1 = Records.newRecord(Priority.class);
- priority1.setPriority(2);
-
- StoredContainerRequest storedContainer1 =
- new StoredContainerRequest(capability, nodes, racks, priority);
- StoredContainerRequest storedContainer2 =
- new StoredContainerRequest(capability, nodes, racks, priority);
- StoredContainerRequest storedContainer3 =
- new StoredContainerRequest(capability, null, null, priority1);
- amClient.addContainerRequest(storedContainer1);
- amClient.addContainerRequest(storedContainer2);
- amClient.addContainerRequest(storedContainer3);
-
- // test addition and storage
- int containersRequestedAny = amClient.remoteRequestsTable.get(priority)
- .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
- assertTrue(containersRequestedAny == 2);
- containersRequestedAny = amClient.remoteRequestsTable.get(priority1)
- .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
- assertTrue(containersRequestedAny == 1);
- List extends Collection> matches =
- amClient.getMatchingRequests(priority, node, capability);
- verifyMatches(matches, 2);
- matches = amClient.getMatchingRequests(priority, rack, capability);
- verifyMatches(matches, 2);
- matches =
- amClient.getMatchingRequests(priority, ResourceRequest.ANY, capability);
- verifyMatches(matches, 2);
- matches = amClient.getMatchingRequests(priority1, rack, capability);
- assertTrue(matches.isEmpty());
- matches =
- amClient.getMatchingRequests(priority1, ResourceRequest.ANY, capability);
- verifyMatches(matches, 1);
-
- // test removal
- amClient.removeContainerRequest(storedContainer3);
- matches = amClient.getMatchingRequests(priority, node, capability);
- verifyMatches(matches, 2);
- amClient.removeContainerRequest(storedContainer2);
- matches = amClient.getMatchingRequests(priority, node, capability);
- verifyMatches(matches, 1);
- matches = amClient.getMatchingRequests(priority, rack, capability);
- verifyMatches(matches, 1);
-
- // test matching of containers
- StoredContainerRequest storedRequest = matches.get(0).iterator().next();
- assertTrue(storedContainer1 == storedRequest);
- amClient.removeContainerRequest(storedContainer1);
- matches =
- amClient.getMatchingRequests(priority, ResourceRequest.ANY, capability);
- assertTrue(matches.isEmpty());
- matches =
- amClient.getMatchingRequests(priority1, ResourceRequest.ANY, capability);
- assertTrue(matches.isEmpty());
- // 0 requests left. everything got cleaned up
- assertTrue(amClient.remoteRequestsTable.isEmpty());
-
- // go through an exemplary allocation, matching and release cycle
- amClient.addContainerRequest(storedContainer1);
- amClient.addContainerRequest(storedContainer3);
- // RM should allocate container within 2 calls to allocate()
- int allocatedContainerCount = 0;
- int iterationsLeft = 2;
- while (allocatedContainerCount < 2
- && iterationsLeft-- > 0) {
- AllocateResponse allocResponse = amClient.allocate(0.1f);
- assertTrue(amClient.ask.size() == 0);
- assertTrue(amClient.release.size() == 0);
-
- assertTrue(nodeCount == amClient.getClusterNodeCount());
- allocatedContainerCount += allocResponse.getAllocatedContainers().size();
- for(Container container : allocResponse.getAllocatedContainers()) {
- ContainerRequest expectedRequest =
- container.getPriority().equals(storedContainer1.getPriority()) ?
- storedContainer1 : storedContainer3;
- matches = amClient.getMatchingRequests(container.getPriority(),
- ResourceRequest.ANY,
- container.getResource());
- // test correct matched container is returned
- verifyMatches(matches, 1);
- ContainerRequest matchedRequest = matches.get(0).iterator().next();
- assertTrue(matchedRequest == expectedRequest);
-
- // assign this container, use it and release it
- amClient.releaseAssignedContainer(container.getId());
- }
- if(allocatedContainerCount < containersRequestedAny) {
- // sleep to let NM's heartbeat to RM and trigger allocations
- sleep(1000);
- }
- }
-
- assertTrue(allocatedContainerCount == 2);
- assertTrue(amClient.release.size() == 2);
- assertTrue(amClient.ask.size() == 0);
- AllocateResponse allocResponse = amClient.allocate(0.1f);
- assertTrue(amClient.release.size() == 0);
- assertTrue(amClient.ask.size() == 0);
- assertTrue(allocResponse.getAllocatedContainers().size() == 0);
-
-
- amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
- null, null);
-
- } finally {
- if (amClient != null && amClient.getServiceState() == STATE.STARTED) {
- amClient.stop();
- }
- }
- }
-
- @Test (timeout=60000)
- public void testAMRMClient() throws YarnException, IOException {
- AMRMClientImpl amClient = null;
- try {
- // start am rm client
- amClient = new AMRMClientImpl(attemptId);
- amClient.init(conf);
- amClient.start();
-
- amClient.registerApplicationMaster("Host", 10000, "");
-
- testAllocation(amClient);
-
- amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
- null, null);
-
- } finally {
- if (amClient != null && amClient.getServiceState() == STATE.STARTED) {
- amClient.stop();
- }
- }
- }
-
- private void testAllocation(final AMRMClientImpl amClient)
- throws YarnException, IOException {
- // setup container request
-
- assertTrue(amClient.ask.size() == 0);
- assertTrue(amClient.release.size() == 0);
-
- amClient.addContainerRequest(
- new ContainerRequest(capability, nodes, racks, priority, 1));
- amClient.addContainerRequest(
- new ContainerRequest(capability, nodes, racks, priority, 3));
- amClient.removeContainerRequest(
- new ContainerRequest(capability, nodes, racks, priority, 2));
-
- int containersRequestedNode = amClient.remoteRequestsTable.get(priority)
- .get(node).get(capability).remoteRequest.getNumContainers();
- int containersRequestedRack = amClient.remoteRequestsTable.get(priority)
- .get(rack).get(capability).remoteRequest.getNumContainers();
- int containersRequestedAny = amClient.remoteRequestsTable.get(priority)
- .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
-
- assertTrue(containersRequestedNode == 2);
- assertTrue(containersRequestedRack == 2);
- assertTrue(containersRequestedAny == 2);
- assertTrue(amClient.ask.size() == 3);
- assertTrue(amClient.release.size() == 0);
-
- // RM should allocate container within 2 calls to allocate()
- int allocatedContainerCount = 0;
- int iterationsLeft = 2;
- Set releases = new TreeSet();
- while (allocatedContainerCount < containersRequestedAny
- && iterationsLeft-- > 0) {
- AllocateResponse allocResponse = amClient.allocate(0.1f);
- assertTrue(amClient.ask.size() == 0);
- assertTrue(amClient.release.size() == 0);
-
- assertTrue(nodeCount == amClient.getClusterNodeCount());
- allocatedContainerCount += allocResponse.getAllocatedContainers().size();
- for(Container container : allocResponse.getAllocatedContainers()) {
- ContainerId rejectContainerId = container.getId();
- releases.add(rejectContainerId);
- amClient.releaseAssignedContainer(rejectContainerId);
- }
- if(allocatedContainerCount < containersRequestedAny) {
- // sleep to let NM's heartbeat to RM and trigger allocations
- sleep(1000);
- }
- }
-
- assertTrue(allocatedContainerCount == containersRequestedAny);
- assertTrue(amClient.release.size() == 2);
- assertTrue(amClient.ask.size() == 0);
-
- // need to tell the AMRMClient that we dont need these resources anymore
- amClient.removeContainerRequest(
- new ContainerRequest(capability, nodes, racks, priority, 2));
- assertTrue(amClient.ask.size() == 3);
- // send 0 container count request for resources that are no longer needed
- ResourceRequest snoopRequest = amClient.ask.iterator().next();
- assertTrue(snoopRequest.getNumContainers() == 0);
-
- // test RPC exception handling
- amClient.addContainerRequest(new ContainerRequest(capability, nodes,
- racks, priority, 2));
- snoopRequest = amClient.ask.iterator().next();
- assertTrue(snoopRequest.getNumContainers() == 2);
-
- AMRMProtocol realRM = amClient.rmClient;
- try {
- AMRMProtocol mockRM = mock(AMRMProtocol.class);
- when(mockRM.allocate(any(AllocateRequest.class))).thenAnswer(
- new Answer() {
- public AllocateResponse answer(InvocationOnMock invocation)
- throws Exception {
- amClient.removeContainerRequest(
- new ContainerRequest(capability, nodes,
- racks, priority, 2));
- throw new Exception();
- }
- });
- amClient.rmClient = mockRM;
- amClient.allocate(0.1f);
- }catch (Exception ioe) {}
- finally {
- amClient.rmClient = realRM;
- }
-
- assertTrue(amClient.release.size() == 2);
- assertTrue(amClient.ask.size() == 3);
- snoopRequest = amClient.ask.iterator().next();
- // verify that the remove request made in between makeRequest and allocate
- // has not been lost
- assertTrue(snoopRequest.getNumContainers() == 0);
-
- iterationsLeft = 2;
- // do a few iterations to ensure RM is not going send new containers
- while(!releases.isEmpty() || iterationsLeft-- > 0) {
- // inform RM of rejection
- AllocateResponse allocResponse = amClient.allocate(0.1f);
- // RM did not send new containers because AM does not need any
- assertTrue(allocResponse.getAllocatedContainers().size() == 0);
- if(allocResponse.getCompletedContainersStatuses().size() > 0) {
- for(ContainerStatus cStatus :allocResponse
- .getCompletedContainersStatuses()) {
- if(releases.contains(cStatus.getContainerId())) {
- assertTrue(cStatus.getState() == ContainerState.COMPLETE);
- assertTrue(cStatus.getExitStatus() == -100);
- releases.remove(cStatus.getContainerId());
- }
- }
- }
- if(iterationsLeft > 0) {
- // sleep to make sure NM's heartbeat
- sleep(1000);
- }
- }
-
- assertTrue(amClient.ask.size() == 0);
- assertTrue(amClient.release.size() == 0);
- }
-
- private void sleep(int sleepTime) {
- try {
- Thread.sleep(sleepTime);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java
deleted file mode 100644
index c5c687c..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java
+++ /dev/null
@@ -1,320 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.client;
-
-import static org.mockito.Matchers.anyFloat;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-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.AllocateResponse;
-import org.apache.hadoop.yarn.api.records.AMCommand;
-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;
-import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-public class TestAMRMClientAsync {
-
- private static final Log LOG = LogFactory.getLog(TestAMRMClientAsync.class);
-
- @SuppressWarnings("unchecked")
- @Test(timeout=10000)
- public void testAMRMClientAsync() throws Exception {
- Configuration conf = new Configuration();
- final AtomicBoolean heartbeatBlock = new AtomicBoolean(true);
- List completed1 = Arrays.asList(
- ContainerStatus.newInstance(newContainerId(0, 0, 0, 0),
- ContainerState.COMPLETE, "", 0));
- List allocated1 = Arrays.asList(
- Container.newInstance(null, null, null, null, null, null));
- final AllocateResponse response1 = createAllocateResponse(
- new ArrayList(), allocated1);
- final AllocateResponse response2 = createAllocateResponse(completed1,
- new ArrayList());
- final AllocateResponse emptyResponse = createAllocateResponse(
- new ArrayList(), new ArrayList());
-
- TestCallbackHandler callbackHandler = new TestCallbackHandler();
- final AMRMClient client = mock(AMRMClientImpl.class);
- final AtomicInteger secondHeartbeatSync = new AtomicInteger(0);
- when(client.allocate(anyFloat())).thenReturn(response1).thenAnswer(new Answer() {
- @Override
- public AllocateResponse answer(InvocationOnMock invocation)
- throws Throwable {
- secondHeartbeatSync.incrementAndGet();
- while(heartbeatBlock.get()) {
- synchronized(heartbeatBlock) {
- heartbeatBlock.wait();
- }
- }
- secondHeartbeatSync.incrementAndGet();
- return response2;
- }
- }).thenReturn(emptyResponse);
- when(client.registerApplicationMaster(anyString(), anyInt(), anyString()))
- .thenReturn(null);
- when(client.getClusterAvailableResources()).thenAnswer(new Answer() {
- @Override
- public Resource answer(InvocationOnMock invocation)
- throws Throwable {
- // take client lock to simulate behavior of real impl
- synchronized (client) {
- Thread.sleep(10);
- }
- return null;
- }
- });
-
- AMRMClientAsync asyncClient =
- new AMRMClientAsync(client, 20, callbackHandler);
- asyncClient.init(conf);
- asyncClient.start();
- asyncClient.registerApplicationMaster("localhost", 1234, null);
-
- // while the CallbackHandler will still only be processing the first response,
- // heartbeater thread should still be sending heartbeats.
- // To test this, wait for the second heartbeat to be received.
- while (secondHeartbeatSync.get() < 1) {
- Thread.sleep(10);
- }
-
- // heartbeat will be blocked. make sure we can call client methods at this
- // time. Checks that heartbeat is not holding onto client lock
- assert(secondHeartbeatSync.get() < 2);
- asyncClient.getClusterAvailableResources();
- // method returned. now unblock heartbeat
- assert(secondHeartbeatSync.get() < 2);
- synchronized (heartbeatBlock) {
- heartbeatBlock.set(false);
- heartbeatBlock.notifyAll();
- }
-
- // allocated containers should come before completed containers
- Assert.assertEquals(null, callbackHandler.takeCompletedContainers());
-
- // wait for the allocated containers from the first heartbeat's response
- while (callbackHandler.takeAllocatedContainers() == null) {
- Assert.assertEquals(null, callbackHandler.takeCompletedContainers());
- Thread.sleep(10);
- }
-
- // wait for the completed containers from the second heartbeat's response
- while (callbackHandler.takeCompletedContainers() == null) {
- Thread.sleep(10);
- }
-
- asyncClient.stop();
-
- Assert.assertEquals(null, callbackHandler.takeAllocatedContainers());
- Assert.assertEquals(null, callbackHandler.takeCompletedContainers());
- }
-
- @Test(timeout=10000)
- public void testAMRMClientAsyncException() throws Exception {
- Configuration conf = new Configuration();
- TestCallbackHandler callbackHandler = new TestCallbackHandler();
- @SuppressWarnings("unchecked")
- AMRMClient client = mock(AMRMClientImpl.class);
- String exStr = "TestException";
- YarnException mockException = mock(YarnException.class);
- when(mockException.getMessage()).thenReturn(exStr);
- when(client.allocate(anyFloat())).thenThrow(mockException);
-
- AMRMClientAsync asyncClient =
- new AMRMClientAsync(client, 20, callbackHandler);
- asyncClient.init(conf);
- asyncClient.start();
-
- synchronized (callbackHandler.notifier) {
- asyncClient.registerApplicationMaster("localhost", 1234, null);
- while(callbackHandler.savedException == null) {
- try {
- callbackHandler.notifier.wait();
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- }
- }
-
- Assert.assertTrue(callbackHandler.savedException.getMessage().contains(exStr));
-
- asyncClient.stop();
- // stopping should have joined all threads and completed all callbacks
- Assert.assertTrue(callbackHandler.callbackCount == 0);
- }
-
- @Test//(timeout=10000)
- public void testAMRMClientAsyncReboot() throws Exception {
- Configuration conf = new Configuration();
- TestCallbackHandler callbackHandler = new TestCallbackHandler();
- @SuppressWarnings("unchecked")
- AMRMClient client = mock(AMRMClientImpl.class);
-
- final AllocateResponse rebootResponse = createAllocateResponse(
- new ArrayList(), new ArrayList());
- rebootResponse.setAMCommand(AMCommand.AM_RESYNC);
- when(client.allocate(anyFloat())).thenReturn(rebootResponse);
-
- AMRMClientAsync asyncClient =
- new AMRMClientAsync(client, 20, callbackHandler);
- asyncClient.init(conf);
- asyncClient.start();
-
- synchronized (callbackHandler.notifier) {
- asyncClient.registerApplicationMaster("localhost", 1234, null);
- while(callbackHandler.reboot == false) {
- try {
- callbackHandler.notifier.wait();
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- }
- }
-
- asyncClient.stop();
- // stopping should have joined all threads and completed all callbacks
- Assert.assertTrue(callbackHandler.callbackCount == 0);
- }
-
- private AllocateResponse createAllocateResponse(
- List completed, List allocated) {
- AllocateResponse response = AllocateResponse.newInstance(0, completed, allocated,
- new ArrayList(), null, null, 1, null);
- return response;
- }
-
- public static ContainerId newContainerId(int appId, int appAttemptId,
- long timestamp, int containerId) {
- ApplicationId applicationId = ApplicationId.newInstance(timestamp, appId);
- ApplicationAttemptId applicationAttemptId =
- ApplicationAttemptId.newInstance(applicationId, appAttemptId);
- return ContainerId.newInstance(applicationAttemptId, containerId);
- }
-
- private class TestCallbackHandler implements AMRMClientAsync.CallbackHandler {
- private volatile List completedContainers;
- private volatile List allocatedContainers;
- Exception savedException = null;
- boolean reboot = false;
- Object notifier = new Object();
-
- int callbackCount = 0;
-
- public List takeCompletedContainers() {
- List ret = completedContainers;
- if (ret == null) {
- return null;
- }
- completedContainers = null;
- synchronized (ret) {
- ret.notify();
- }
- return ret;
- }
-
- public List takeAllocatedContainers() {
- List ret = allocatedContainers;
- if (ret == null) {
- return null;
- }
- allocatedContainers = null;
- synchronized (ret) {
- ret.notify();
- }
- return ret;
- }
-
- @Override
- public void onContainersCompleted(List statuses) {
- completedContainers = statuses;
- // wait for containers to be taken before returning
- synchronized (completedContainers) {
- while (completedContainers != null) {
- try {
- completedContainers.wait();
- } catch (InterruptedException ex) {
- LOG.error("Interrupted during wait", ex);
- }
- }
- }
- }
-
- @Override
- public void onContainersAllocated(List containers) {
- allocatedContainers = containers;
- // wait for containers to be taken before returning
- synchronized (allocatedContainers) {
- while (allocatedContainers != null) {
- try {
- allocatedContainers.wait();
- } catch (InterruptedException ex) {
- LOG.error("Interrupted during wait", ex);
- }
- }
- }
- }
-
- @Override
- public void onShutdownRequest() {
- reboot = true;
- synchronized (notifier) {
- notifier.notifyAll();
- }
- }
-
- @Override
- public void onNodesUpdated(List updatedNodes) {}
-
- @Override
- public float getProgress() {
- callbackCount++;
- return 0.5f;
- }
-
- @Override
- public void onError(Exception e) {
- savedException = e;
- synchronized (notifier) {
- notifier.notifyAll();
- }
- }
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java
deleted file mode 100644
index 990368e..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.client;
-
-import java.io.IOException;
-import java.io.PrintStream;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.tools.GetGroupsTestBase;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.service.Service.STATE;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-
-public class TestGetGroups extends GetGroupsTestBase {
-
- private static final Log LOG = LogFactory.getLog(TestGetGroups.class);
-
- private static ResourceManager resourceManager;
-
- private static Configuration conf;
-
- @BeforeClass
- public static void setUpResourceManager() throws IOException, InterruptedException {
- conf = new YarnConfiguration();
- resourceManager = new ResourceManager() {
- @Override
- protected void doSecureLogin() throws IOException {
- };
- };
- resourceManager.init(conf);
- new Thread() {
- public void run() {
- resourceManager.start();
- };
- }.start();
- int waitCount = 0;
- while (resourceManager.getServiceState() == STATE.INITED
- && waitCount++ < 10) {
- LOG.info("Waiting for RM to start...");
- Thread.sleep(1000);
- }
- if (resourceManager.getServiceState() != STATE.STARTED) {
- throw new IOException(
- "ResourceManager failed to start. Final state is "
- + resourceManager.getServiceState());
- }
- LOG.info("ResourceManager RMAdmin address: " +
- conf.get(YarnConfiguration.RM_ADMIN_ADDRESS));
- }
-
- @SuppressWarnings("static-access")
- @Before
- public void setUpConf() {
- super.conf = this.conf;
- }
-
- @AfterClass
- public static void tearDownResourceManager() throws InterruptedException {
- if (resourceManager != null) {
- LOG.info("Stopping ResourceManager...");
- resourceManager.stop();
- }
- }
-
- @Override
- protected Tool getTool(PrintStream o) {
- return new GetGroupsForTesting(conf, o);
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java
deleted file mode 100644
index 6f46ded..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java
+++ /dev/null
@@ -1,324 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.client;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.MiniYARNCluster;
-import org.apache.hadoop.yarn.service.Service.STATE;
-import org.apache.hadoop.yarn.util.Records;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestNMClient {
- Configuration conf = null;
- MiniYARNCluster yarnCluster = null;
- YarnClientImpl yarnClient = null;
- AMRMClientImpl rmClient = null;
- NMClientImpl nmClient = null;
- List nodeReports = null;
- ApplicationAttemptId attemptId = null;
- int nodeCount = 3;
-
- @Before
- public void setup() throws YarnException, IOException {
- // start minicluster
- conf = new YarnConfiguration();
- yarnCluster =
- new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
- yarnCluster.init(conf);
- yarnCluster.start();
- assertNotNull(yarnCluster);
- assertEquals(STATE.STARTED, yarnCluster.getServiceState());
-
- // start rm client
- yarnClient = new YarnClientImpl();
- yarnClient.init(conf);
- yarnClient.start();
- assertNotNull(yarnClient);
- assertEquals(STATE.STARTED, yarnClient.getServiceState());
-
- // get node info
- nodeReports = yarnClient.getNodeReports();
-
- // submit new app
- GetNewApplicationResponse newApp = yarnClient.getNewApplication();
- ApplicationId appId = newApp.getApplicationId();
-
- ApplicationSubmissionContext appContext = Records
- .newRecord(ApplicationSubmissionContext.class);
- // set the application id
- appContext.setApplicationId(appId);
- // set the application name
- appContext.setApplicationName("Test");
- // Set the priority for the application master
- Priority pri = Priority.newInstance(0);
- appContext.setPriority(pri);
- // Set the queue to which this application is to be submitted in the RM
- appContext.setQueue("default");
- // Set up the container launch context for the application master
- ContainerLaunchContext amContainer = Records
- .newRecord(ContainerLaunchContext.class);
- appContext.setAMContainerSpec(amContainer);
- // unmanaged AM
- appContext.setUnmanagedAM(true);
- // Create the request to send to the applications manager
- SubmitApplicationRequest appRequest = Records
- .newRecord(SubmitApplicationRequest.class);
- appRequest.setApplicationSubmissionContext(appContext);
- // Submit the application to the applications manager
- yarnClient.submitApplication(appContext);
-
- // wait for app to start
- int iterationsLeft = 30;
- while (iterationsLeft > 0) {
- ApplicationReport appReport = yarnClient.getApplicationReport(appId);
- if (appReport.getYarnApplicationState() ==
- YarnApplicationState.ACCEPTED) {
- attemptId = appReport.getCurrentApplicationAttemptId();
- break;
- }
- sleep(1000);
- --iterationsLeft;
- }
- if (iterationsLeft == 0) {
- fail("Application hasn't bee started");
- }
-
- // start am rm client
- rmClient = new AMRMClientImpl(attemptId);
- rmClient.init(conf);
- rmClient.start();
- assertNotNull(rmClient);
- assertEquals(STATE.STARTED, rmClient.getServiceState());
-
- // start am nm client
- nmClient = new NMClientImpl();
- nmClient.init(conf);
- nmClient.start();
- assertNotNull(nmClient);
- assertEquals(STATE.STARTED, nmClient.getServiceState());
- }
-
- @After
- public void tearDown() {
- rmClient.stop();
-
- // leave one unclosed
- assertEquals(1, nmClient.startedContainers.size());
- // default true
- assertTrue(nmClient.cleanupRunningContainers.get());
- // don't stop the running containers
- nmClient.cleanupRunningContainersOnStop(false);
- assertFalse(nmClient.cleanupRunningContainers.get());
- nmClient.stop();
- assertTrue(nmClient.startedContainers.size() > 0);
- // stop the running containers
- nmClient.cleanupRunningContainersOnStop(true);
- assertTrue(nmClient.cleanupRunningContainers.get());
- nmClient.stop();
- assertEquals(0, nmClient.startedContainers.size());
-
- yarnClient.stop();
- yarnCluster.stop();
- }
-
- @Test (timeout = 60000)
- public void testNMClient()
- throws YarnException, IOException {
-
- rmClient.registerApplicationMaster("Host", 10000, "");
-
- testContainerManagement(nmClient, allocateContainers(rmClient, 5));
-
- rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
- null, null);
- }
-
- private Set allocateContainers(
- AMRMClientImpl rmClient, int num)
- throws YarnException, IOException {
- // setup container request
- Resource capability = Resource.newInstance(1024, 0);
- Priority priority = Priority.newInstance(0);
- String node = nodeReports.get(0).getNodeId().getHost();
- String rack = nodeReports.get(0).getRackName();
- String[] nodes = new String[] {node};
- String[] racks = new String[] {rack};
-
- for (int i = 0; i < num; ++i) {
- rmClient.addContainerRequest(new ContainerRequest(capability, nodes,
- racks, priority, 1));
- }
-
- int containersRequestedAny = rmClient.remoteRequestsTable.get(priority)
- .get(ResourceRequest.ANY).get(capability).remoteRequest
- .getNumContainers();
-
- // RM should allocate container within 2 calls to allocate()
- int allocatedContainerCount = 0;
- int iterationsLeft = 2;
- Set containers = new TreeSet();
- while (allocatedContainerCount < containersRequestedAny
- && iterationsLeft > 0) {
- AllocateResponse allocResponse = rmClient.allocate(0.1f);
-
- allocatedContainerCount += allocResponse.getAllocatedContainers().size();
- for(Container container : allocResponse.getAllocatedContainers()) {
- containers.add(container);
- }
- if(allocatedContainerCount < containersRequestedAny) {
- // sleep to let NM's heartbeat to RM and trigger allocations
- sleep(1000);
- }
-
- --iterationsLeft;
- }
- return containers;
- }
-
- private void testContainerManagement(NMClientImpl nmClient,
- Set containers) throws YarnException, IOException {
- int size = containers.size();
- int i = 0;
- for (Container container : containers) {
- // getContainerStatus shouldn't be called before startContainer,
- // otherwise, NodeManager cannot find the container
- try {
- nmClient.getContainerStatus(container.getId(), container.getNodeId(),
- container.getContainerToken());
- fail("Exception is expected");
- } catch (YarnException e) {
- assertTrue("The thrown exception is not expected",
- e.getMessage().contains("is not handled by this NodeManager"));
- }
-
- // stopContainer shouldn't be called before startContainer,
- // otherwise, an exception will be thrown
- try {
- nmClient.stopContainer(container.getId(), container.getNodeId(),
- container.getContainerToken());
- fail("Exception is expected");
- } catch (YarnException e) {
- assertTrue("The thrown exception is not expected",
- e.getMessage().contains(
- "is either not started yet or already stopped"));
- }
-
- Credentials ts = new Credentials();
- DataOutputBuffer dob = new DataOutputBuffer();
- ts.writeTokenStorageToStream(dob);
- ByteBuffer securityTokens =
- ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
- ContainerLaunchContext clc =
- Records.newRecord(ContainerLaunchContext.class);
- clc.setTokens(securityTokens);
- try {
- nmClient.startContainer(container, clc);
- } catch (YarnException e) {
- fail("Exception is not expected");
- }
-
- // leave one container unclosed
- if (++i < size) {
- // NodeManager may still need some time to make the container started
- testGetContainerStatus(container, i, ContainerState.RUNNING, "",
- -1000);
-
- try {
- nmClient.stopContainer(container.getId(), container.getNodeId(),
- container.getContainerToken());
- } catch (YarnException e) {
- fail("Exception is not expected");
- }
-
- // getContainerStatus can be called after stopContainer
- testGetContainerStatus(container, i, ContainerState.COMPLETE,
- "Container killed by the ApplicationMaster.", 143);
- }
- }
- }
-
- private void sleep(int sleepTime) {
- try {
- Thread.sleep(sleepTime);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- }
-
- private void testGetContainerStatus(Container container, int index,
- ContainerState state, String diagnostics, int exitStatus)
- throws YarnException, IOException {
- while (true) {
- try {
- ContainerStatus status = nmClient.getContainerStatus(
- container.getId(), container.getNodeId(),
- container.getContainerToken());
- // NodeManager may still need some time to get the stable
- // container status
- if (status.getState() == state) {
- assertEquals(container.getId(), status.getContainerId());
- assertTrue("" + index + ": " + status.getDiagnostics(),
- status.getDiagnostics().contains(diagnostics));
- assertEquals(exitStatus, status.getExitStatus());
- break;
- }
- Thread.sleep(100);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- }
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java
deleted file mode 100644
index 6fff279..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java
+++ /dev/null
@@ -1,543 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.client;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.BrokenBarrierException;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicIntegerArray;
-
-import junit.framework.Assert;
-
-import org.apache.hadoop.conf.Configuration;
-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;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Token;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.ipc.RPCUtil;
-import org.junit.Test;
-
-
-public class TestNMClientAsync {
-
- private final RecordFactory recordFactory =
- RecordFactoryProvider.getRecordFactory(null);
-
- private NMClientAsync asyncClient;
- private NodeId nodeId;
- private Token containerToken;
-
- @Test (timeout = 30000)
- public void testNMClientAsync() throws Exception {
- Configuration conf = new Configuration();
- conf.setInt(YarnConfiguration.NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE, 10);
-
- // Threads to run are more than the max size of the thread pool
- int expectedSuccess = 40;
- int expectedFailure = 40;
-
- asyncClient = new MockNMClientAsync1(expectedSuccess, expectedFailure);
- asyncClient.init(conf);
- Assert.assertEquals("The max thread pool size is not correctly set",
- 10, asyncClient.maxThreadPoolSize);
- asyncClient.start();
-
-
- for (int i = 0; i < expectedSuccess + expectedFailure; ++i) {
- if (i == expectedSuccess) {
- while (!((TestCallbackHandler1) asyncClient.callbackHandler)
- .isAllSuccessCallsExecuted()) {
- Thread.sleep(10);
- }
- asyncClient.client = mockNMClient(1);
- }
- Container container = mockContainer(i);
- ContainerLaunchContext clc =
- recordFactory.newRecordInstance(ContainerLaunchContext.class);
- asyncClient.startContainer(container, clc);
- }
- while (!((TestCallbackHandler1) asyncClient.callbackHandler)
- .isStartAndQueryFailureCallsExecuted()) {
- Thread.sleep(10);
- }
- asyncClient.client = mockNMClient(2);
- ((TestCallbackHandler1) asyncClient.callbackHandler).path = false;
- for (int i = 0; i < expectedFailure; ++i) {
- Container container = mockContainer(
- expectedSuccess + expectedFailure + i);
- ContainerLaunchContext clc =
- recordFactory.newRecordInstance(ContainerLaunchContext.class);
- asyncClient.startContainer(container, clc);
- }
- while (!((TestCallbackHandler1) asyncClient.callbackHandler)
- .isStopFailureCallsExecuted()) {
- Thread.sleep(10);
- }
- for (String errorMsg :
- ((TestCallbackHandler1) asyncClient.callbackHandler).errorMsgs) {
- System.out.println(errorMsg);
- }
- Assert.assertEquals("Error occurs in CallbackHandler", 0,
- ((TestCallbackHandler1) asyncClient.callbackHandler).errorMsgs.size());
- for (String errorMsg : ((MockNMClientAsync1) asyncClient).errorMsgs) {
- System.out.println(errorMsg);
- }
- Assert.assertEquals("Error occurs in ContainerEventProcessor", 0,
- ((MockNMClientAsync1) asyncClient).errorMsgs.size());
- // When the callback functions are all executed, the event processor threads
- // may still not terminate and the containers may still not removed.
- while (asyncClient.containers.size() > 0) {
- Thread.sleep(10);
- }
- asyncClient.stop();
- Assert.assertFalse(
- "The thread of Container Management Event Dispatcher is still alive",
- asyncClient.eventDispatcherThread.isAlive());
- Assert.assertTrue("The thread pool is not shut down",
- asyncClient.threadPool.isShutdown());
- }
-
- private class MockNMClientAsync1 extends NMClientAsync {
- private Set errorMsgs =
- Collections.synchronizedSet(new HashSet());
-
- protected MockNMClientAsync1(int expectedSuccess, int expectedFailure)
- throws YarnException, IOException {
- super(MockNMClientAsync1.class.getName(), mockNMClient(0),
- new TestCallbackHandler1(expectedSuccess, expectedFailure));
- }
-
- private class MockContainerEventProcessor extends ContainerEventProcessor {
- public MockContainerEventProcessor(ContainerEvent event) {
- super(event);
- }
-
- @Override
- public void run() {
- try {
- super.run();
- } catch (RuntimeException e) {
- // If the unexpected throwable comes from error callback functions, it
- // will break ContainerEventProcessor.run(). Therefore, monitor
- // the exception here
- errorMsgs.add("Unexpected throwable from callback functions should" +
- " be ignored by Container " + event.getContainerId());
- }
- }
- }
-
- @Override
- protected ContainerEventProcessor getContainerEventProcessor(
- ContainerEvent event) {
- return new MockContainerEventProcessor(event);
- }
- }
-
- private class TestCallbackHandler1
- implements NMClientAsync.CallbackHandler {
-
- private boolean path = true;
-
- private int expectedSuccess;
- private int expectedFailure;
-
- private AtomicInteger actualStartSuccess = new AtomicInteger(0);
- private AtomicInteger actualStartFailure = new AtomicInteger(0);
- private AtomicInteger actualQuerySuccess = new AtomicInteger(0);
- private AtomicInteger actualQueryFailure = new AtomicInteger(0);
- private AtomicInteger actualStopSuccess = new AtomicInteger(0);
- private AtomicInteger actualStopFailure = new AtomicInteger(0);
-
- private AtomicIntegerArray actualStartSuccessArray;
- private AtomicIntegerArray actualStartFailureArray;
- private AtomicIntegerArray actualQuerySuccessArray;
- private AtomicIntegerArray actualQueryFailureArray;
- private AtomicIntegerArray actualStopSuccessArray;
- private AtomicIntegerArray actualStopFailureArray;
-
- private Set errorMsgs =
- Collections.synchronizedSet(new HashSet());
-
- public TestCallbackHandler1(int expectedSuccess, int expectedFailure) {
- this.expectedSuccess = expectedSuccess;
- this.expectedFailure = expectedFailure;
-
- actualStartSuccessArray = new AtomicIntegerArray(expectedSuccess);
- actualStartFailureArray = new AtomicIntegerArray(expectedFailure);
- actualQuerySuccessArray = new AtomicIntegerArray(expectedSuccess);
- actualQueryFailureArray = new AtomicIntegerArray(expectedFailure);
- actualStopSuccessArray = new AtomicIntegerArray(expectedSuccess);
- actualStopFailureArray = new AtomicIntegerArray(expectedFailure);
- }
-
- @Override
- public void onContainerStarted(ContainerId containerId,
- Map allServiceResponse) {
- if (path) {
- if (containerId.getId() >= expectedSuccess) {
- errorMsgs.add("Container " + containerId +
- " should throw the exception onContainerStarted");
- return;
- }
- actualStartSuccess.addAndGet(1);
- actualStartSuccessArray.set(containerId.getId(), 1);
-
- // move on to the following success tests
- asyncClient.getContainerStatus(containerId, nodeId, containerToken);
- } else {
- // move on to the following failure tests
- asyncClient.stopContainer(containerId, nodeId, containerToken);
- }
-
- // Shouldn't crash the test thread
- throw new RuntimeException("Ignorable Exception");
- }
-
- @Override
- public void onContainerStatusReceived(ContainerId containerId,
- ContainerStatus containerStatus) {
- if (containerId.getId() >= expectedSuccess) {
- errorMsgs.add("Container " + containerId +
- " should throw the exception onContainerStatusReceived");
- return;
- }
- actualQuerySuccess.addAndGet(1);
- actualQuerySuccessArray.set(containerId.getId(), 1);
- // move on to the following success tests
- asyncClient.stopContainer(containerId, nodeId, containerToken);
-
- // Shouldn't crash the test thread
- throw new RuntimeException("Ignorable Exception");
- }
-
- @Override
- public void onContainerStopped(ContainerId containerId) {
- if (containerId.getId() >= expectedSuccess) {
- errorMsgs.add("Container " + containerId +
- " should throw the exception onContainerStopped");
- return;
- }
- actualStopSuccess.addAndGet(1);
- actualStopSuccessArray.set(containerId.getId(), 1);
-
- // Shouldn't crash the test thread
- throw new RuntimeException("Ignorable Exception");
- }
-
- @Override
- public void onStartContainerError(ContainerId containerId, Throwable t) {
- // If the unexpected throwable comes from success callback functions, it
- // will be handled by the error callback functions. Therefore, monitor
- // the exception here
- if (t instanceof RuntimeException) {
- errorMsgs.add("Unexpected throwable from callback functions should be" +
- " ignored by Container " + containerId);
- }
- if (containerId.getId() < expectedSuccess) {
- errorMsgs.add("Container " + containerId +
- " shouldn't throw the exception onStartContainerError");
- return;
- }
- actualStartFailure.addAndGet(1);
- actualStartFailureArray.set(containerId.getId() - expectedSuccess, 1);
- // move on to the following failure tests
- asyncClient.getContainerStatus(containerId, nodeId, containerToken);
-
- // Shouldn't crash the test thread
- throw new RuntimeException("Ignorable Exception");
- }
-
- @Override
- public void onStopContainerError(ContainerId containerId, Throwable t) {
- if (t instanceof RuntimeException) {
- errorMsgs.add("Unexpected throwable from callback functions should be" +
- " ignored by Container " + containerId);
- }
- if (containerId.getId() < expectedSuccess + expectedFailure) {
- errorMsgs.add("Container " + containerId +
- " shouldn't throw the exception onStopContainerError");
- return;
- }
-
- actualStopFailure.addAndGet(1);
- actualStopFailureArray.set(
- containerId.getId() - expectedSuccess - expectedFailure, 1);
-
- // Shouldn't crash the test thread
- throw new RuntimeException("Ignorable Exception");
- }
-
- @Override
- public void onGetContainerStatusError(ContainerId containerId,
- Throwable t) {
- if (t instanceof RuntimeException) {
- errorMsgs.add("Unexpected throwable from callback functions should be"
- + " ignored by Container " + containerId);
- }
- if (containerId.getId() < expectedSuccess) {
- errorMsgs.add("Container " + containerId +
- " shouldn't throw the exception onGetContainerStatusError");
- return;
- }
- actualQueryFailure.addAndGet(1);
- actualQueryFailureArray.set(containerId.getId() - expectedSuccess, 1);
-
- // Shouldn't crash the test thread
- throw new RuntimeException("Ignorable Exception");
- }
-
- public boolean isAllSuccessCallsExecuted() {
- boolean isAllSuccessCallsExecuted =
- actualStartSuccess.get() == expectedSuccess &&
- actualQuerySuccess.get() == expectedSuccess &&
- actualStopSuccess.get() == expectedSuccess;
- if (isAllSuccessCallsExecuted) {
- assertAtomicIntegerArray(actualStartSuccessArray);
- assertAtomicIntegerArray(actualQuerySuccessArray);
- assertAtomicIntegerArray(actualStopSuccessArray);
- }
- return isAllSuccessCallsExecuted;
- }
-
- public boolean isStartAndQueryFailureCallsExecuted() {
- boolean isStartAndQueryFailureCallsExecuted =
- actualStartFailure.get() == expectedFailure &&
- actualQueryFailure.get() == expectedFailure;
- if (isStartAndQueryFailureCallsExecuted) {
- assertAtomicIntegerArray(actualStartFailureArray);
- assertAtomicIntegerArray(actualQueryFailureArray);
- }
- return isStartAndQueryFailureCallsExecuted;
- }
-
- public boolean isStopFailureCallsExecuted() {
- boolean isStopFailureCallsExecuted =
- actualStopFailure.get() == expectedFailure;
- if (isStopFailureCallsExecuted) {
- assertAtomicIntegerArray(actualStopFailureArray);
- }
- return isStopFailureCallsExecuted;
- }
-
- private void assertAtomicIntegerArray(AtomicIntegerArray array) {
- for (int i = 0; i < array.length(); ++i) {
- Assert.assertEquals(1, array.get(i));
- }
- }
- }
-
- private NMClient mockNMClient(int mode)
- throws YarnException, IOException {
- NMClient client = mock(NMClient.class);
- switch (mode) {
- case 0:
- when(client.startContainer(any(Container.class),
- any(ContainerLaunchContext.class))).thenReturn(
- Collections.emptyMap());
- when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class),
- any(Token.class))).thenReturn(
- recordFactory.newRecordInstance(ContainerStatus.class));
- doNothing().when(client).stopContainer(any(ContainerId.class),
- any(NodeId.class), any(Token.class));
- break;
- case 1:
- doThrow(RPCUtil.getRemoteException("Start Exception")).when(client)
- .startContainer(any(Container.class),
- any(ContainerLaunchContext.class));
- doThrow(RPCUtil.getRemoteException("Query Exception")).when(client)
- .getContainerStatus(any(ContainerId.class), any(NodeId.class),
- any(Token.class));
- doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client)
- .stopContainer(any(ContainerId.class), any(NodeId.class),
- any(Token.class));
- break;
- case 2:
- when(client.startContainer(any(Container.class),
- any(ContainerLaunchContext.class))).thenReturn(
- Collections.emptyMap());
- when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class),
- any(Token.class))).thenReturn(
- recordFactory.newRecordInstance(ContainerStatus.class));
- doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client)
- .stopContainer(any(ContainerId.class), any(NodeId.class),
- any(Token.class));
- }
- return client;
- }
-
- @Test (timeout = 10000)
- public void testOutOfOrder() throws Exception {
- CyclicBarrier barrierA = new CyclicBarrier(2);
- CyclicBarrier barrierB = new CyclicBarrier(2);
- CyclicBarrier barrierC = new CyclicBarrier(2);
- asyncClient = new MockNMClientAsync2(barrierA, barrierB, barrierC);
- asyncClient.init(new Configuration());
- asyncClient.start();
-
- final Container container = mockContainer(1);
- final ContainerLaunchContext clc =
- recordFactory.newRecordInstance(ContainerLaunchContext.class);
-
- // start container from another thread
- Thread t = new Thread() {
- @Override
- public void run() {
- asyncClient.startContainer(container, clc);
- }
- };
- t.start();
-
- barrierA.await();
- asyncClient.stopContainer(container.getId(), container.getNodeId(),
- container.getContainerToken());
- barrierC.await();
-
- Assert.assertFalse("Starting and stopping should be out of order",
- ((TestCallbackHandler2) asyncClient.callbackHandler)
- .exceptionOccurred.get());
- }
-
- private class MockNMClientAsync2 extends NMClientAsync {
- private CyclicBarrier barrierA;
- private CyclicBarrier barrierB;
-
- protected MockNMClientAsync2(CyclicBarrier barrierA, CyclicBarrier barrierB,
- CyclicBarrier barrierC) throws YarnException, IOException {
- super(MockNMClientAsync2.class.getName(), mockNMClient(0),
- new TestCallbackHandler2(barrierC));
- this.barrierA = barrierA;
- this.barrierB = barrierB;
- }
-
- private class MockContainerEventProcessor extends ContainerEventProcessor {
-
- public MockContainerEventProcessor(ContainerEvent event) {
- super(event);
- }
-
- @Override
- public void run() {
- try {
- if (event.getType() == ContainerEventType.START_CONTAINER) {
- barrierA.await();
- barrierB.await();
- }
- super.run();
- if (event.getType() == ContainerEventType.STOP_CONTAINER) {
- barrierB.await();
- }
- } catch (InterruptedException e) {
- e.printStackTrace();
- } catch (BrokenBarrierException e) {
- e.printStackTrace();
- }
- }
- }
-
- @Override
- protected ContainerEventProcessor getContainerEventProcessor(
- ContainerEvent event) {
- return new MockContainerEventProcessor(event);
- }
- }
-
- private class TestCallbackHandler2
- implements NMClientAsync.CallbackHandler {
- private CyclicBarrier barrierC;
- private AtomicBoolean exceptionOccurred = new AtomicBoolean(false);
-
- public TestCallbackHandler2(CyclicBarrier barrierC) {
- this.barrierC = barrierC;
- }
-
- @Override
- public void onContainerStarted(ContainerId containerId,
- Map allServiceResponse) {
- }
-
- @Override
- public void onContainerStatusReceived(ContainerId containerId,
- ContainerStatus containerStatus) {
- }
-
- @Override
- public void onContainerStopped(ContainerId containerId) {
- }
-
- @Override
- public void onStartContainerError(ContainerId containerId, Throwable t) {
- if (!t.getMessage().equals(NMClientAsync.StatefulContainer
- .OutOfOrderTransition.STOP_BEFORE_START_ERROR_MSG)) {
- exceptionOccurred.set(true);
- return;
- }
- try {
- barrierC.await();
- } catch (InterruptedException e) {
- e.printStackTrace();
- } catch (BrokenBarrierException e) {
- e.printStackTrace();
- }
- }
-
- @Override
- public void onGetContainerStatusError(ContainerId containerId,
- Throwable t) {
- }
-
- @Override
- public void onStopContainerError(ContainerId containerId, Throwable t) {
- }
-
- }
-
- private Container mockContainer(int i) {
- ApplicationId appId =
- ApplicationId.newInstance(System.currentTimeMillis(), 1);
- ApplicationAttemptId attemptId =
- ApplicationAttemptId.newInstance(appId, 1);
- ContainerId containerId = ContainerId.newInstance(attemptId, i);
- nodeId = NodeId.newInstance("localhost", 0);
- // Create an empty record
- containerToken = recordFactory.newRecordInstance(Token.class);
- return Container.newInstance(containerId, nodeId, null, null, null,
- containerToken);
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java
deleted file mode 100644
index fbc876a..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements. See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership. The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License. You may obtain a copy of the License at
-*
-* http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.hadoop.yarn.client;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.util.HashMap;
-
-import junit.framework.Assert;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.ClientRMProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.log4j.Level;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-import org.junit.Test;
-
-public class TestYarnClient {
-
- @Test
- public void test() {
- // More to come later.
- }
-
- @Test
- public void testClientStop() {
- Configuration conf = new Configuration();
- ResourceManager rm = new ResourceManager();
- rm.init(conf);
- rm.start();
-
- YarnClient client = new YarnClientImpl();
- client.init(conf);
- client.start();
- client.stop();
- }
-
- @Test (timeout = 30000)
- public void testSubmitApplication() {
- Configuration conf = new Configuration();
- conf.setLong(YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS,
- 100); // speed up tests
- final YarnClient client = new MockYarnClient();
- client.init(conf);
- client.start();
-
- YarnApplicationState[] exitStates = new YarnApplicationState[]
- {
- YarnApplicationState.SUBMITTED,
- YarnApplicationState.ACCEPTED,
- YarnApplicationState.RUNNING,
- YarnApplicationState.FINISHED,
- YarnApplicationState.FAILED,
- YarnApplicationState.KILLED
- };
- for (int i = 0; i < exitStates.length; ++i) {
- ApplicationSubmissionContext context =
- mock(ApplicationSubmissionContext.class);
- ApplicationId applicationId = ApplicationId.newInstance(
- System.currentTimeMillis(), i);
- when(context.getApplicationId()).thenReturn(applicationId);
- ((MockYarnClient) client).setYarnApplicationState(exitStates[i]);
- try {
- client.submitApplication(context);
- } catch (YarnException e) {
- Assert.fail("Exception is not expected.");
- } catch (IOException e) {
- Assert.fail("Exception is not expected.");
- }
- verify(((MockYarnClient) client).mockReport,times(4 * i + 4))
- .getYarnApplicationState();
- }
-
- client.stop();
- }
-
- @Test(timeout = 30000)
- public void testApplicationType() throws Exception {
- Logger rootLogger = LogManager.getRootLogger();
- rootLogger.setLevel(Level.DEBUG);
- MockRM rm = new MockRM();
- rm.start();
- RMApp app = rm.submitApp(2000);
- RMApp app1 =
- rm.submitApp(200, "name", "user",
- new HashMap(), false, "default", -1,
- null, "MAPREDUCE");
- Assert.assertEquals("YARN", app.getApplicationType());
- Assert.assertEquals("MAPREDUCE", app1.getApplicationType());
- rm.stop();
- }
-
- @Test(timeout = 30000)
- public void testApplicationTypeLimit() throws Exception {
- Logger rootLogger = LogManager.getRootLogger();
- rootLogger.setLevel(Level.DEBUG);
- MockRM rm = new MockRM();
- rm.start();
- RMApp app1 =
- rm.submitApp(200, "name", "user",
- new HashMap(), false, "default", -1,
- null, "MAPREDUCE-LENGTH-IS-20");
- Assert.assertEquals("MAPREDUCE-LENGTH-IS-", app1.getApplicationType());
- rm.stop();
- }
-
- private static class MockYarnClient extends YarnClientImpl {
- private ApplicationReport mockReport;
-
- public MockYarnClient() {
- super();
- }
-
- @Override
- public void start() {
- rmClient = mock(ClientRMProtocol.class);
- GetApplicationReportResponse mockResponse =
- mock(GetApplicationReportResponse.class);
- mockReport = mock(ApplicationReport.class);
- try{
- when(rmClient.getApplicationReport(any(
- GetApplicationReportRequest.class))).thenReturn(mockResponse);
- } catch (YarnException e) {
- Assert.fail("Exception is not expected.");
- } catch (IOException e) {
- Assert.fail("Exception is not expected.");
- }
- when(mockResponse.getApplicationReport()).thenReturn(mockReport);
- }
-
- @Override
- public void stop() {
- }
-
- public void setYarnApplicationState(YarnApplicationState state) {
- when(mockReport.getYarnApplicationState()).thenReturn(
- YarnApplicationState.NEW, YarnApplicationState.NEW_SAVING,
- YarnApplicationState.NEW_SAVING, state);
- }
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index d71d193..fa8972e 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -111,6 +111,13 @@
+ Whether ApplicationMasterLauncher use NMClient to start or
+ stop containers.
+ yarn.resourcemanager.amlauncher.using.nmclient
+ false
+
+
+
Are acls enabled.
yarn.acl.enable
true
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
index 06f58a8..82386f0 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
@@ -43,6 +43,10 @@
org.apache.hadoop
+ hadoop-yarn-client
+
+
+ org.apache.hadoop
hadoop-hdfs
test-jar
test
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncherViaNMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncherViaNMClient.java
new file mode 100644
index 0000000..1b866e7
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncherViaNMClient.java
@@ -0,0 +1,170 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.resourcemanager.amlauncher;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import javax.crypto.SecretKey;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+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.security.ApplicationTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+
+/**
+ * The launch of the AM itself.
+ */
+public class AMLauncherViaNMClient implements Runnable {
+
+ private static final Log LOG = LogFactory.getLog(AMLauncher.class);
+
+ private final ApplicationMasterLauncher amLaunchService;
+ private final RMAppAttempt application;
+ private final AMLauncherEventType eventType;
+ private final RMContext rmContext;
+ private final Container masterContainer;
+
+ public AMLauncherViaNMClient(
+ ApplicationMasterLauncher amLaunchService, RMContext rmContext,
+ RMAppAttempt application, AMLauncherEventType eventType) {
+ this.amLaunchService = amLaunchService;
+ this.application = application;
+ this.eventType = eventType;
+ this.rmContext = rmContext;
+ this.masterContainer = application.getMasterContainer();
+ }
+
+ private ContainerLaunchContext createAMContainerLaunchContext(
+ ApplicationSubmissionContext applicationMasterContext,
+ ContainerId containerID) throws IOException {
+
+ // Construct the actual Container
+ ContainerLaunchContext container =
+ applicationMasterContext.getAMContainerSpec();
+ LOG.info("Command to launch container "
+ + containerID
+ + " : "
+ + StringUtils.arrayToString(container.getCommands().toArray(
+ new String[0])));
+
+ // Finalize the container
+ setupTokensAndEnv(container, containerID);
+
+ return container;
+ }
+
+ private void setupTokensAndEnv(
+ ContainerLaunchContext container, ContainerId containerID)
+ throws IOException {
+ Map environment = container.getEnvironment();
+ environment.put(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV,
+ application.getWebProxyBase());
+ // Set AppSubmitTime and MaxAppAttempts to be consumable by the AM.
+ ApplicationId applicationId =
+ application.getAppAttemptId().getApplicationId();
+ environment.put(
+ ApplicationConstants.APP_SUBMIT_TIME_ENV,
+ String.valueOf(rmContext.getRMApps()
+ .get(applicationId)
+ .getSubmitTime()));
+ environment.put(ApplicationConstants.MAX_APP_ATTEMPTS_ENV,
+ String.valueOf(rmContext.getRMApps().get(
+ applicationId).getMaxAppAttempts()));
+
+ if (UserGroupInformation.isSecurityEnabled()) {
+ // TODO: Security enabled/disabled info should come from RM.
+
+ Credentials credentials = new Credentials();
+
+ DataInputByteBuffer dibb = new DataInputByteBuffer();
+ if (container.getTokens() != null) {
+ // TODO: Don't do this kind of checks everywhere.
+ dibb.reset(container.getTokens());
+ credentials.readTokenStorageStream(dibb);
+ }
+
+ // Add application token
+ Token applicationToken =
+ application.getApplicationToken();
+ if(applicationToken != null) {
+ credentials.addToken(applicationToken.getService(), applicationToken);
+ }
+ DataOutputBuffer dob = new DataOutputBuffer();
+ credentials.writeTokenStorageToStream(dob);
+ container.setTokens(ByteBuffer.wrap(dob.getData(), 0,
+ dob.getLength()));
+
+ SecretKey clientSecretKey =
+ this.rmContext.getClientToAMTokenSecretManager().getMasterKey(
+ application.getAppAttemptId());
+ String encoded =
+ Base64.encodeBase64URLSafeString(clientSecretKey.getEncoded());
+ environment.put(
+ ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME,
+ encoded);
+ }
+ }
+
+ public void run() {
+ switch (eventType) {
+ case LAUNCH:
+ LOG.info("Launching master" + application.getAppAttemptId());
+ ContainerId masterContainerId = masterContainer.getId();
+ ApplicationSubmissionContext applicationContext =
+ application.getSubmissionContext();
+ LOG.info("Setting up container " + masterContainer
+ + " for AM " + application.getAppAttemptId());
+ ContainerLaunchContext launchContext;
+ try {
+ launchContext = createAMContainerLaunchContext(
+ applicationContext, masterContainerId);
+ amLaunchService.startContainer(masterContainer, launchContext);
+ } catch (IOException e) {
+ // createAMContainerLaunchContext can throw exception as well
+ amLaunchService.getAMContainerListener().onStartContainerError(
+ masterContainerId, e);
+ }
+ break;
+ case CLEANUP:
+ LOG.info("Cleaning master " + application.getAppAttemptId());
+ amLaunchService.stopContainer(masterContainer.getId(),
+ masterContainer.getNodeId(), masterContainer.getContainerToken());
+ break;
+ default:
+ LOG.warn("Received unknown event-type " + eventType + ". Ignoring.");
+ break;
+ }
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
index a114d80..d70929b 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
@@ -17,6 +17,8 @@
*/
package org.apache.hadoop.yarn.server.resourcemanager.amlauncher;
+import java.nio.ByteBuffer;
+import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
@@ -24,13 +26,26 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.client.NMClient;
+import org.apache.hadoop.yarn.client.NMClientAsync;
+import org.apache.hadoop.yarn.client.NMClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
-import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
+import com.google.common.annotations.VisibleForTesting;
-public class ApplicationMasterLauncher extends AbstractService implements
+
+public class ApplicationMasterLauncher extends NMClientAsync implements
EventHandler {
private static final Log LOG = LogFactory.getLog(
ApplicationMasterLauncher.class);
@@ -41,15 +56,31 @@
= new LinkedBlockingQueue();
protected final RMContext context;
-
+
+ protected boolean usingNMClient;
+
public ApplicationMasterLauncher(RMContext context) {
- super(ApplicationMasterLauncher.class.getName());
+ this(context, new NMClientImpl());
+ }
+
+ @Private
+ @VisibleForTesting
+ public ApplicationMasterLauncher(RMContext context, NMClient nmClient) {
+ super(ApplicationMasterLauncher.class.getName(), nmClient,
+ new AMContainerListener(context));
this.context = context;
this.launcherPool = new ThreadPoolExecutor(10, 10, 1,
TimeUnit.HOURS, new LinkedBlockingQueue());
this.launcherHandlingThread = new LauncherThread();
}
-
+
+ public void init(Configuration conf) {
+ usingNMClient = conf.getBoolean(
+ YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT,
+ YarnConfiguration.DEFAULT_RM_AMLAUNCHER_USING_NMCLIENT);
+ super.init(conf);
+ }
+
public void start() {
launcherHandlingThread.start();
super.start();
@@ -57,8 +88,12 @@ public void start() {
protected Runnable createRunnableLauncher(RMAppAttempt application,
AMLauncherEventType event) {
- Runnable launcher =
- new AMLauncher(context, application, event, getConfig());
+ Runnable launcher;
+ if (usingNMClient) {
+ launcher = new AMLauncherViaNMClient(this, context, application, event);
+ } else {
+ launcher = new AMLauncher(context, application, event, getConfig());
+ }
return launcher;
}
@@ -120,4 +155,63 @@ public synchronized void handle(AMLauncherEvent appEvent) {
break;
}
}
+
+ protected CallbackHandler getAMContainerListener() {
+ return callbackHandler;
+ }
+
+ private static class AMContainerListener implements CallbackHandler {
+
+ private final RMContext context;
+
+ public AMContainerListener(RMContext context) {
+ this.context = context;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void onContainerStarted(ContainerId containerId,
+ Map allServiceResponse) {
+ LOG.info("Done launching container " + containerId
+ + " for AM " + containerId.getApplicationAttemptId());
+ context.getDispatcher().getEventHandler().handle(
+ new RMAppAttemptEvent(containerId.getApplicationAttemptId(),
+ RMAppAttemptEventType.LAUNCHED));
+ }
+
+ @Override
+ public void onContainerStatusReceived(ContainerId containerId,
+ ContainerStatus containerStatus) {
+ }
+
+ @Override
+ public void onContainerStopped(ContainerId containerId) {
+ LOG.info(
+ "Done cleaning master " + containerId.getApplicationAttemptId());
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void onStartContainerError(ContainerId containerId, Throwable t) {
+ String message = "Error launching" +
+ containerId.getApplicationAttemptId() + ". Got exception: " +
+ StringUtils.stringifyException(t);
+ LOG.info(message);
+ context.getDispatcher().getEventHandler().handle(
+ new RMAppAttemptLaunchFailedEvent(
+ containerId.getApplicationAttemptId(), message));
+ }
+
+ @Override
+ public void onGetContainerStatusError(
+ ContainerId containerId, Throwable t) {
+ }
+
+ @Override
+ public void onStopContainerError(ContainerId containerId, Throwable t) {
+ LOG.info(
+ "Error cleaning master " + containerId.getApplicationAttemptId(), t);
+ }
+
+ }
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMWithCustomAMLauncher.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMWithCustomAMLauncher.java
index 9a4526a..139897a 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMWithCustomAMLauncher.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMWithCustomAMLauncher.java
@@ -21,19 +21,32 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.ContainerManager;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.client.NMClient;
+import org.apache.hadoop.yarn.server.resourcemanager.TestApplicationMasterLauncher.MyContainerManagerAndNMClient;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherViaNMClient;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
public class MockRMWithCustomAMLauncher extends MockRM {
- private final ContainerManager containerManager;
+ protected final ContainerManager containerManager;
+
+ protected NMClient nmClient;
+
+ public MockRMWithCustomAMLauncher() {
+ this((ContainerManager) null);
+ }
public MockRMWithCustomAMLauncher(ContainerManager containerManager) {
this(new Configuration(), containerManager);
}
+ public MockRMWithCustomAMLauncher(Configuration conf) {
+ this(conf, null);
+ }
+
public MockRMWithCustomAMLauncher(Configuration conf,
ContainerManager containerManager) {
super(conf);
@@ -42,18 +55,38 @@ public MockRMWithCustomAMLauncher(Configuration conf,
@Override
protected ApplicationMasterLauncher createAMLauncher() {
- return new ApplicationMasterLauncher(getRMContext()) {
+ if (nmClient == null) {
+ nmClient = new MyContainerManagerAndNMClient();
+ }
+ return new ApplicationMasterLauncher(getRMContext(), nmClient) {
+
@Override
protected Runnable createRunnableLauncher(RMAppAttempt application,
AMLauncherEventType event) {
- return new AMLauncher(context, application, event, getConfig()) {
- @Override
- protected ContainerManager getContainerMgrProxy(
- ContainerId containerId) {
- return containerManager;
- }
- };
+ Runnable launcher;
+ if (usingNMClient) {
+ launcher =
+ new AMLauncherViaNMClient(this, context, application, event);
+ } else {
+ launcher = new AMLauncher(context, application, event, getConfig()) {
+ @Override
+ protected ContainerManager getContainerMgrProxy(
+ ContainerId containerId) {
+ return containerManager;
+ }
+ };
+ }
+ return launcher;
}
};
}
+
+ public NMClient getMyNMClient() {
+ return nmClient;
+ }
+
+ public ContainerManager getMyContainerManager() {
+ return containerManager;
+ }
+
}
\ No newline at end of file
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
index d25418d..013b1d9 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
@@ -22,6 +22,7 @@
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.security.PrivilegedAction;
+import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@@ -44,13 +45,26 @@
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
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.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.client.NMClient;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherViaNMClient;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
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.utils.BuilderUtils;
+import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.util.Records;
import org.junit.Assert;
import org.junit.Test;
@@ -67,11 +81,13 @@
UserGroupInformation.setConfiguration(confWithSecurityEnabled);
}
- public static final class MyContainerManager implements ContainerManager {
+ public static final class MyContainerManagerAndNMClient
+ extends AbstractService implements ContainerManager, NMClient {
public ByteBuffer amTokens;
- public MyContainerManager() {
+ public MyContainerManagerAndNMClient() {
+ super(MyContainerManagerAndNMClient.class.getName());
}
@Override
@@ -79,27 +95,54 @@ public MyContainerManager() {
startContainer(StartContainerRequest request)
throws YarnException {
amTokens = request.getContainerLaunchContext().getTokens();
- return null;
+ return StartContainerResponse.newInstance(null);
}
@Override
public StopContainerResponse stopContainer(StopContainerRequest request)
throws YarnException {
- // TODO Auto-generated method stub
- return null;
+ return StopContainerResponse.newInstance();
}
@Override
public GetContainerStatusResponse getContainerStatus(
GetContainerStatusRequest request) throws YarnException {
- // TODO Auto-generated method stub
- return null;
+ return GetContainerStatusResponse.newInstance(null);
+ }
+
+ @Override
+ public Map startContainer(Container container,
+ ContainerLaunchContext containerLaunchContext) throws YarnException,
+ IOException {
+ amTokens = containerLaunchContext.getTokens();
+ return Collections.emptyMap();
+ }
+
+ @Override
+ public void stopContainer(ContainerId containerId, NodeId nodeId,
+ Token containerToken) throws YarnException, IOException {
+ }
+
+ @Override
+ public ContainerStatus getContainerStatus(ContainerId containerId,
+ NodeId nodeId, Token containerToken) throws YarnException, IOException {
+ return ContainerStatus.newInstance(
+ containerId, ContainerState.NEW, "", -1);
+ }
+
+ @Override
+ public void cleanupRunningContainersOnStop(boolean enabled) {
}
}
public static class MockRMWithAMS extends MockRMWithCustomAMLauncher {
- public MockRMWithAMS(Configuration conf, ContainerManager containerManager) {
+ public MockRMWithAMS(Configuration conf) {
+ super(conf);
+ }
+
+ public MockRMWithAMS(Configuration conf,
+ ContainerManager containerManager) {
super(conf, containerManager);
}
@@ -112,13 +155,63 @@ protected void doSecureLogin() throws IOException {
protected ApplicationMasterService createApplicationMasterService() {
return new ApplicationMasterService(getRMContext(), this.scheduler);
}
+
+ @Override
+ protected ApplicationMasterLauncher createAMLauncher() {
+ if (nmClient == null) {
+ nmClient = new MyContainerManagerAndNMClient();
+ }
+ return new ApplicationMasterLauncher(getRMContext(), nmClient) {
+
+ @Override
+ protected Runnable createRunnableLauncher(RMAppAttempt application,
+ AMLauncherEventType event) {
+ Runnable launcher;
+ if (usingNMClient) {
+ launcher =
+ new AMLauncherViaNMClient(this, context, application, event);
+ } else {
+ launcher =
+ new AMLauncher(context, application, event, getConfig()) {
+ @Override
+ protected ContainerManager getContainerMgrProxy(
+ ContainerId containerId) {
+ return containerManager;
+ }
+ };
+ }
+ return launcher;
+ }
+ };
+ }
}
@Test
public void testAuthorizedAccess() throws Exception {
- MyContainerManager containerManager = new MyContainerManager();
- final MockRM rm =
- new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+ MyContainerManagerAndNMClient cmAndNMClient =
+ new MyContainerManagerAndNMClient();
+ confWithSecurityEnabled.setBoolean(
+ YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, false);
+ final MockRMWithAMS rm =
+ new MockRMWithAMS(confWithSecurityEnabled, cmAndNMClient);
+ verifyAuthorizedAccess(rm, cmAndNMClient);
+ }
+
+ @Test
+ public void testAuthorizedAccessWithNMClient() throws Exception {
+ confWithSecurityEnabled.setBoolean(
+ YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, true);
+ final MockRMWithAMS rm =
+ new MockRMWithAMS(confWithSecurityEnabled);
+ Assert.assertTrue(
+ rm.getMyNMClient() instanceof MyContainerManagerAndNMClient);
+ MyContainerManagerAndNMClient cmAndNMClient =
+ (MyContainerManagerAndNMClient) rm.getMyNMClient();
+ verifyAuthorizedAccess(rm, cmAndNMClient);
+ }
+
+ private void verifyAuthorizedAccess(final MockRMWithAMS rm,
+ MyContainerManagerAndNMClient cmAndNMClient) throws Exception {
rm.start();
MockNM nm1 = rm.registerNode("localhost:1234", 5120);
@@ -131,11 +224,11 @@ public void testAuthorizedAccess() throws Exception {
nm1.nodeHeartbeat(true);
int waitCount = 0;
- while (containerManager.amTokens == null && waitCount++ < 20) {
+ while (cmAndNMClient.amTokens == null && waitCount++ < 20) {
LOG.info("Waiting for AM Launch to happen..");
Thread.sleep(1000);
}
- Assert.assertNotNull(containerManager.amTokens);
+ Assert.assertNotNull(cmAndNMClient.amTokens);
RMAppAttempt attempt = app.getCurrentAppAttempt();
ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -149,8 +242,8 @@ public void testAuthorizedAccess() throws Exception {
.createRemoteUser(applicationAttemptId.toString());
Credentials credentials = new Credentials();
DataInputByteBuffer buf = new DataInputByteBuffer();
- containerManager.amTokens.rewind();
- buf.reset(containerManager.amTokens);
+ cmAndNMClient.amTokens.rewind();
+ buf.reset(cmAndNMClient.amTokens);
credentials.readTokenStorageStream(buf);
currentUser.addCredentials(credentials);
@@ -175,9 +268,15 @@ public AMRMProtocol run() {
}
@Test
- public void testUnauthorizedAccess() throws Exception {
- MyContainerManager containerManager = new MyContainerManager();
- MockRM rm = new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+ public void testUnauthorizedAccessWithNMClient() throws Exception {
+ confWithSecurityEnabled.setBoolean(
+ YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, true);
+ final MockRMWithAMS rm =
+ new MockRMWithAMS(confWithSecurityEnabled);
+ Assert.assertTrue(
+ rm.getMyNMClient() instanceof MyContainerManagerAndNMClient);
+ MyContainerManagerAndNMClient cmAndNMClient =
+ (MyContainerManagerAndNMClient) rm.getMyNMClient();
rm.start();
MockNM nm1 = rm.registerNode("localhost:1234", 5120);
@@ -187,11 +286,11 @@ public void testUnauthorizedAccess() throws Exception {
nm1.nodeHeartbeat(true);
int waitCount = 0;
- while (containerManager.amTokens == null && waitCount++ < 20) {
+ while (cmAndNMClient.amTokens == null && waitCount++ < 20) {
LOG.info("Waiting for AM Launch to happen..");
Thread.sleep(1000);
}
- Assert.assertNotNull(containerManager.amTokens);
+ Assert.assertNotNull(cmAndNMClient.amTokens);
RMAppAttempt attempt = app.getCurrentAppAttempt();
ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -233,8 +332,8 @@ public AMRMProtocol run() {
// Now try to validate invalid authorization.
Credentials credentials = new Credentials();
DataInputByteBuffer buf = new DataInputByteBuffer();
- containerManager.amTokens.rewind();
- buf.reset(containerManager.amTokens);
+ cmAndNMClient.amTokens.rewind();
+ buf.reset(cmAndNMClient.amTokens);
credentials.readTokenStorageStream(buf);
currentUser.addCredentials(credentials);
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
index 9231442..52b79a8 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
@@ -19,10 +19,13 @@
package org.apache.hadoop.yarn.server.resourcemanager;
import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.ContainerManager;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
@@ -32,9 +35,14 @@
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
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.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.client.NMClient;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.ipc.RPCUtil;
@@ -43,6 +51,7 @@
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.utils.BuilderUtils;
+import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -54,8 +63,12 @@
private static final Log LOG = LogFactory
.getLog(TestApplicationMasterLauncher.class);
- private static final class MyContainerManagerImpl implements
- ContainerManager {
+ protected static final class MyContainerManagerAndNMClient
+ extends AbstractService implements ContainerManager, NMClient {
+
+ public MyContainerManagerAndNMClient() {
+ super(MyContainerManagerAndNMClient.class.getName());
+ }
boolean launched = false;
boolean cleanedup = false;
@@ -69,12 +82,55 @@
public StartContainerResponse
startContainer(StartContainerRequest request)
throws YarnException {
- LOG.info("Container started by MyContainerManager: " + request);
+ startContainer(request.getContainerToken(),
+ request.getContainerLaunchContext());
+ return StartContainerResponse.newInstance(null);
+ }
+
+ @Override
+ public StopContainerResponse stopContainer(StopContainerRequest request)
+ throws YarnException {
+ stopContainer();
+ return StopContainerResponse.newInstance();
+ }
+
+ @Override
+ public GetContainerStatusResponse getContainerStatus(
+ GetContainerStatusRequest request) throws YarnException {
+ return GetContainerStatusResponse.newInstance(null);
+ }
+
+ @Override
+ public Map startContainer(Container container,
+ ContainerLaunchContext containerLaunchContext) throws YarnException,
+ IOException {
+ startContainer(container.getContainerToken(), containerLaunchContext);
+ return Collections.emptyMap();
+ }
+
+ @Override
+ public void stopContainer(ContainerId containerId, NodeId nodeId,
+ Token containerToken) throws YarnException, IOException {
+ stopContainer();
+ }
+
+ @Override
+ public ContainerStatus getContainerStatus(ContainerId containerId,
+ NodeId nodeId, Token containerToken) throws YarnException, IOException {
+ return ContainerStatus.newInstance(
+ containerId, ContainerState.NEW, "", -1);
+ }
+
+ @Override
+ public void cleanupRunningContainersOnStop(boolean enabled) {
+ }
+
+ private void startContainer(Token containerToken,
+ ContainerLaunchContext containerLaunchContext) throws YarnException {
+ LOG.info("Container started by MyContainerManagerAndNMClient");
launched = true;
- Map env =
- request.getContainerLaunchContext().getEnvironment();
+ Map env = containerLaunchContext.getEnvironment();
- Token containerToken = request.getContainerToken();
ContainerTokenIdentifier tokenId = null;
try {
@@ -92,32 +148,43 @@
Long.parseLong(env.get(ApplicationConstants.APP_SUBMIT_TIME_ENV));
maxAppAttempts =
Integer.parseInt(env.get(ApplicationConstants.MAX_APP_ATTEMPTS_ENV));
- return null;
}
- @Override
- public StopContainerResponse stopContainer(StopContainerRequest request)
- throws YarnException {
- LOG.info("Container cleaned up by MyContainerManager");
+ private void stopContainer() {
+ LOG.info("Container cleaned up by MyContainerManagerAndNMClient");
cleanedup = true;
- return null;
}
-
- @Override
- public GetContainerStatusResponse getContainerStatus(
- GetContainerStatusRequest request) throws YarnException {
- return null;
- }
-
}
@Test
public void testAMLaunchAndCleanup() throws Exception {
Logger rootLogger = LogManager.getRootLogger();
rootLogger.setLevel(Level.DEBUG);
- MyContainerManagerImpl containerManager = new MyContainerManagerImpl();
- MockRMWithCustomAMLauncher rm = new MockRMWithCustomAMLauncher(
- containerManager);
+ Configuration conf = new Configuration();
+ conf.setBoolean(YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, false);
+ MyContainerManagerAndNMClient cmAndNMClient =
+ new MyContainerManagerAndNMClient();
+ MockRMWithCustomAMLauncher rm =
+ new MockRMWithCustomAMLauncher(conf, cmAndNMClient);
+ verifyAMLaunchAndCleanup(rm, cmAndNMClient);
+ }
+
+ @Test
+ public void testAMLaunchAndCleanupWithNMClient() throws Exception {
+ Logger rootLogger = LogManager.getRootLogger();
+ rootLogger.setLevel(Level.DEBUG);
+ Configuration conf = new Configuration();
+ conf.setBoolean(YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, true);
+ MockRMWithCustomAMLauncher rm = new MockRMWithCustomAMLauncher(conf);
+ Assert.assertTrue(
+ rm.getMyNMClient() instanceof MyContainerManagerAndNMClient);
+ MyContainerManagerAndNMClient cmAndNMClient =
+ (MyContainerManagerAndNMClient) rm.getMyNMClient();
+ verifyAMLaunchAndCleanup(rm, cmAndNMClient);
+ }
+
+ private void verifyAMLaunchAndCleanup(MockRMWithCustomAMLauncher rm,
+ MyContainerManagerAndNMClient cmAndNMClient) throws Exception {
rm.start();
MockNM nm1 = rm.registerNode("127.0.0.1:1234", 5120);
@@ -127,25 +194,25 @@ public void testAMLaunchAndCleanup() throws Exception {
nm1.nodeHeartbeat(true);
int waitCount = 0;
- while (containerManager.launched == false && waitCount++ < 20) {
+ while (cmAndNMClient.launched == false && waitCount++ < 20) {
LOG.info("Waiting for AM Launch to happen..");
Thread.sleep(1000);
}
- Assert.assertTrue(containerManager.launched);
+ Assert.assertTrue(cmAndNMClient.launched);
RMAppAttempt attempt = app.getCurrentAppAttempt();
ApplicationAttemptId appAttemptId = attempt.getAppAttemptId();
Assert.assertEquals(appAttemptId.toString(),
- containerManager.attemptIdAtContainerManager);
+ cmAndNMClient.attemptIdAtContainerManager);
Assert.assertEquals(app.getSubmitTime(),
- containerManager.submitTimeAtContainerManager);
+ cmAndNMClient.submitTimeAtContainerManager);
Assert.assertEquals(app.getRMAppAttempt(appAttemptId)
.getMasterContainer().getId()
- .toString(), containerManager.containerIdAtContainerManager);
+ .toString(), cmAndNMClient.containerIdAtContainerManager);
Assert.assertEquals(nm1.getNodeId().toString(),
- containerManager.nmHostAtContainerManager);
+ cmAndNMClient.nmHostAtContainerManager);
Assert.assertEquals(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS,
- containerManager.maxAppAttempts);
+ cmAndNMClient.maxAppAttempts);
MockAM am = new MockAM(rm.getRMContext(), rm
.getApplicationMasterService(), appAttemptId);
@@ -157,11 +224,11 @@ public void testAMLaunchAndCleanup() throws Exception {
am.waitForState(RMAppAttemptState.FINISHED);
waitCount = 0;
- while (containerManager.cleanedup == false && waitCount++ < 20) {
+ while (cmAndNMClient.cleanedup == false && waitCount++ < 20) {
LOG.info("Waiting for AM Cleanup to happen..");
Thread.sleep(1000);
}
- Assert.assertTrue(containerManager.cleanedup);
+ Assert.assertTrue(cmAndNMClient.cleanedup);
am.waitForState(RMAppAttemptState.FINISHED);
rm.stop();
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
index 2227948..cbac0e8 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
@@ -44,9 +44,8 @@
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
-import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
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.TestAMAuthorization.MyContainerManagerAndNMClient;
import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
import org.apache.hadoop.yarn.server.resourcemanager.resource.DominantResourceCalculator;
import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
@@ -266,13 +265,28 @@ public void testValidateResourceRequest() {
// expected
}
}
-
+
@Test
public void testValidateResourceBlacklistRequest() throws Exception {
+ Configuration conf = new YarnConfiguration();
+ conf.setBoolean(YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, false);
+ MyContainerManagerAndNMClient cmAndNMClient =
+ new MyContainerManagerAndNMClient();
+ MockRMWithAMS rm = new MockRMWithAMS(conf, cmAndNMClient);
+ verifyValidateResourceBlacklistRequest(rm);
+ }
+
+ @Test
+ public void testValidateResourceBlacklistRequestWithNMClient()
+ throws Exception {
+ Configuration conf = new YarnConfiguration();
+ conf.setBoolean(YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, true);
+ MockRMWithAMS rm = new MockRMWithAMS(conf);
+ verifyValidateResourceBlacklistRequest(rm);
+ }
- MyContainerManager containerManager = new MyContainerManager();
- final MockRM rm =
- new MockRMWithAMS(new YarnConfiguration(), containerManager);
+ private void verifyValidateResourceBlacklistRequest(
+ final MockRMWithAMS rm) throws Exception {
rm.start();
MockNM nm1 = rm.registerNode("localhost:1234", 5120);
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java
index 4fb81c6..4bb26f8 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java
@@ -35,11 +35,12 @@
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
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.TestAMAuthorization.MyContainerManagerAndNMClient;
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.utils.BuilderUtils;
@@ -62,15 +63,40 @@
/**
* Validate that application tokens are unusable after the
* application-finishes.
- *
+ *
* @throws Exception
*/
@Test
public void testTokenExpiry() throws Exception {
+ MyContainerManagerAndNMClient cmAndNMClient =
+ new MyContainerManagerAndNMClient();
+ confWithSecurityEnabled.setBoolean(
+ YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, false);
+ final MockRMWithAMS rm =
+ new MockRMWithAMS(confWithSecurityEnabled, cmAndNMClient);
+ verifyTokenExpiry(rm, cmAndNMClient);
+ }
+
+ /**
+ * Validate that application tokens are unusable after the
+ * application-finishes.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testTokenExpiryWithNMClient() throws Exception {
+ confWithSecurityEnabled.setBoolean(
+ YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, true);
+ final MockRMWithAMS rm = new MockRMWithAMS(confWithSecurityEnabled);
+ Assert.assertTrue(
+ rm.getMyNMClient() instanceof MyContainerManagerAndNMClient);
+ MyContainerManagerAndNMClient cmAndNMClient =
+ (MyContainerManagerAndNMClient) rm.getMyNMClient();
+ verifyTokenExpiry(rm, cmAndNMClient);
+ }
- MyContainerManager containerManager = new MyContainerManager();
- final MockRM rm =
- new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+ private void verifyTokenExpiry(MockRMWithAMS rm,
+ MyContainerManagerAndNMClient cmAndNMClient) throws Exception {
rm.start();
final Configuration conf = rm.getConfig();
@@ -85,11 +111,11 @@ public void testTokenExpiry() throws Exception {
nm1.nodeHeartbeat(true);
int waitCount = 0;
- while (containerManager.amTokens == null && waitCount++ < 20) {
+ while (cmAndNMClient.amTokens == null && waitCount++ < 20) {
LOG.info("Waiting for AM Launch to happen..");
Thread.sleep(1000);
}
- Assert.assertNotNull(containerManager.amTokens);
+ Assert.assertNotNull(cmAndNMClient.amTokens);
RMAppAttempt attempt = app.getCurrentAppAttempt();
ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -100,8 +126,8 @@ public void testTokenExpiry() throws Exception {
.createRemoteUser(applicationAttemptId.toString());
Credentials credentials = new Credentials();
DataInputByteBuffer buf = new DataInputByteBuffer();
- containerManager.amTokens.rewind();
- buf.reset(containerManager.amTokens);
+ cmAndNMClient.amTokens.rewind();
+ buf.reset(cmAndNMClient.amTokens);
credentials.readTokenStorageStream(buf);
currentUser.addCredentials(credentials);
@@ -154,15 +180,40 @@ public void testTokenExpiry() throws Exception {
/**
* Validate master-key-roll-over and that tokens are usable even after
* master-key-roll-over.
- *
+ *
* @throws Exception
*/
@Test
public void testMasterKeyRollOver() throws Exception {
+ MyContainerManagerAndNMClient cmAndNMClient =
+ new MyContainerManagerAndNMClient();
+ confWithSecurityEnabled.setBoolean(
+ YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, false);
+ final MockRMWithAMS rm =
+ new MockRMWithAMS(confWithSecurityEnabled, cmAndNMClient);
+ verifyMasterKeyRollOver(rm, cmAndNMClient);
+ }
+
+ /**
+ * Validate master-key-roll-over and that tokens are usable even after
+ * master-key-roll-over.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testMasterKeyRollOverWithNMClient() throws Exception {
+ confWithSecurityEnabled.setBoolean(
+ YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, true);
+ final MockRMWithAMS rm = new MockRMWithAMS(confWithSecurityEnabled);
+ Assert.assertTrue(
+ rm.getMyNMClient() instanceof MyContainerManagerAndNMClient);
+ MyContainerManagerAndNMClient cmAndNMClient =
+ (MyContainerManagerAndNMClient) rm.getMyNMClient();
+ verifyMasterKeyRollOver(rm, cmAndNMClient);
+ }
- MyContainerManager containerManager = new MyContainerManager();
- final MockRM rm =
- new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+ private void verifyMasterKeyRollOver(MockRMWithAMS rm,
+ MyContainerManagerAndNMClient cmAndNMClient) throws Exception {
rm.start();
final Configuration conf = rm.getConfig();
@@ -177,11 +228,11 @@ public void testMasterKeyRollOver() throws Exception {
nm1.nodeHeartbeat(true);
int waitCount = 0;
- while (containerManager.amTokens == null && waitCount++ < 20) {
+ while (cmAndNMClient.amTokens == null && waitCount++ < 20) {
LOG.info("Waiting for AM Launch to happen..");
Thread.sleep(1000);
}
- Assert.assertNotNull(containerManager.amTokens);
+ Assert.assertNotNull(cmAndNMClient.amTokens);
RMAppAttempt attempt = app.getCurrentAppAttempt();
ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -192,8 +243,8 @@ public void testMasterKeyRollOver() throws Exception {
.createRemoteUser(applicationAttemptId.toString());
Credentials credentials = new Credentials();
DataInputByteBuffer buf = new DataInputByteBuffer();
- containerManager.amTokens.rewind();
- buf.reset(containerManager.amTokens);
+ cmAndNMClient.amTokens.rewind();
+ buf.reset(cmAndNMClient.amTokens);
credentials.readTokenStorageStream(buf);
currentUser.addCredentials(credentials);
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java
index 883286d..56f9333 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java
@@ -23,7 +23,10 @@
import java.io.IOException;
import java.lang.annotation.Annotation;
import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
import java.security.PrivilegedExceptionAction;
+import java.util.Collections;
+import java.util.Map;
import javax.security.sasl.SaslException;
@@ -57,6 +60,14 @@
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
+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;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.client.NMClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -65,9 +76,13 @@
import org.apache.hadoop.yarn.security.client.ClientTokenSelector;
import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
-import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.MockRMWithCustomAMLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherViaNMClient;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
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.utils.BuilderUtils;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.util.ProtoUtils;
@@ -149,7 +164,12 @@ public synchronized void start() {
}
}
- private static class CustomNM implements ContainerManager {
+ private static class CustomContainerManagerAndNMClient
+ extends AbstractService implements ContainerManager, NMClient {
+
+ public CustomContainerManagerAndNMClient() {
+ super(CustomContainerManagerAndNMClient.class.getName());
+ }
public String clientTokensSecret;
@@ -159,35 +179,88 @@ public StartContainerResponse startContainer(StartContainerRequest request)
this.clientTokensSecret =
request.getContainerLaunchContext().getEnvironment()
.get(ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME);
- return null;
+ return StartContainerResponse.newInstance(null);
}
@Override
public StopContainerResponse stopContainer(StopContainerRequest request)
throws YarnException {
- return null;
+ return StopContainerResponse.newInstance();
}
@Override
public GetContainerStatusResponse getContainerStatus(
GetContainerStatusRequest request) throws YarnException {
- return null;
+ return GetContainerStatusResponse.newInstance(null);
+ }
+
+ @Override
+ public Map startContainer(Container container,
+ ContainerLaunchContext containerLaunchContext) throws YarnException,
+ IOException {
+ this.clientTokensSecret = containerLaunchContext.getEnvironment()
+ .get(ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME);
+ return Collections.emptyMap();
+ }
+
+ @Override
+ public void stopContainer(ContainerId containerId, NodeId nodeId,
+ org.apache.hadoop.yarn.api.records.Token containerToken)
+ throws YarnException, IOException {
+ }
+
+ @Override
+ public ContainerStatus getContainerStatus(ContainerId containerId,
+ NodeId nodeId, org.apache.hadoop.yarn.api.records.Token containerToken)
+ throws YarnException, IOException {
+ return ContainerStatus.newInstance(
+ containerId, ContainerState.NEW, "", -1);
+ }
+
+ @Override
+ public void cleanupRunningContainersOnStop(boolean enabled) {
}
}
@Test
public void testClientTokens() throws Exception {
-
final Configuration conf = new Configuration();
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
"kerberos");
+ conf.setBoolean(YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, false);
UserGroupInformation.setConfiguration(conf);
+ DrainDispatcher dispatcher = new DrainDispatcher();
+ CustomContainerManagerAndNMClient cmAndNMClient =
+ new CustomContainerManagerAndNMClient();
+ MockRMWithCustomAMLauncher rm =
+ createMockRMWithCustomAMLauncher(dispatcher, cmAndNMClient, conf);
+ verifyClientTokens(conf, dispatcher, rm, cmAndNMClient);
+ }
- CustomNM containerManager = new CustomNM();
- final DrainDispatcher dispatcher = new DrainDispatcher();
+ @Test
+ public void testClientTokensWithNMClient() throws Exception {
+ final Configuration conf = new Configuration();
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+ "kerberos");
+ conf.setBoolean(YarnConfiguration.RM_AMLAUNCHER_USING_NMCLIENT, true);
+ UserGroupInformation.setConfiguration(conf);
+ DrainDispatcher dispatcher = new DrainDispatcher();
+ MockRMWithCustomAMLauncher rm =
+ createMockRMWithCustomAMLauncher(dispatcher, null, conf);
+ Assert.assertTrue(
+ rm.getMyNMClient() instanceof CustomContainerManagerAndNMClient);
+ CustomContainerManagerAndNMClient cmAndNMClient =
+ (CustomContainerManagerAndNMClient) rm.getMyNMClient();
+ verifyClientTokens(conf, dispatcher, rm, cmAndNMClient);
+ }
- MockRM rm = new MockRMWithCustomAMLauncher(conf, containerManager) {
+ private MockRMWithCustomAMLauncher createMockRMWithCustomAMLauncher(
+ final Dispatcher dispatcher,
+ CustomContainerManagerAndNMClient cmAndNMClient, Configuration conf) {
+ MockRMWithCustomAMLauncher rm = new MockRMWithCustomAMLauncher(
+ conf, cmAndNMClient) {
+ @Override
protected ClientRMService createClientRMService() {
return new ClientRMService(this.rmContext, scheduler,
this.rmAppManager, this.applicationACLsManager,
@@ -202,7 +275,42 @@ protected Dispatcher createDispatcher() {
@Override
protected void doSecureLogin() throws IOException {
}
+
+ @Override
+ protected ApplicationMasterLauncher createAMLauncher() {
+ if (nmClient == null) {
+ nmClient = new CustomContainerManagerAndNMClient();
+ }
+ return new ApplicationMasterLauncher(getRMContext(), nmClient) {
+
+ @Override
+ protected Runnable createRunnableLauncher(RMAppAttempt application,
+ AMLauncherEventType event) {
+ Runnable launcher;
+ if (usingNMClient) {
+ launcher =
+ new AMLauncherViaNMClient(this, context, application, event);
+ } else {
+ launcher =
+ new AMLauncher(context, application, event, getConfig()) {
+ @Override
+ protected ContainerManager getContainerMgrProxy(
+ ContainerId containerId) {
+ return containerManager;
+ }
+ };
+ }
+ return launcher;
+ }
+ };
+ }
};
+ return rm;
+ }
+
+ private void verifyClientTokens(final Configuration conf,
+ DrainDispatcher dispatcher, MockRMWithCustomAMLauncher rm,
+ CustomContainerManagerAndNMClient cmAndNMClient) throws Exception {
rm.start();
// Submit an app
@@ -225,17 +333,17 @@ protected void doSecureLogin() throws IOException {
// Wait till AM is 'launched'
int waitTime = 0;
- while (containerManager.clientTokensSecret == null && waitTime++ < 20) {
+ while (cmAndNMClient.clientTokensSecret == null && waitTime++ < 20) {
Thread.sleep(1000);
}
- Assert.assertNotNull(containerManager.clientTokensSecret);
+ Assert.assertNotNull(cmAndNMClient.clientTokensSecret);
// Start the AM with the correct shared-secret.
ApplicationAttemptId appAttemptId =
app.getAppAttempts().keySet().iterator().next();
Assert.assertNotNull(appAttemptId);
final CustomAM am =
- new CustomAM(appAttemptId, containerManager.clientTokensSecret);
+ new CustomAM(appAttemptId, cmAndNMClient.clientTokensSecret);
am.init(conf);
am.start();
@@ -320,6 +428,8 @@ public Void run() throws Exception {
return null;
}
});
+
+ rm.stop();
}
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
index 600c647..112a79c 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
@@ -50,6 +50,11 @@
test-jar
test
+
+ org.apache.hadoop
+ hadoop-yarn-client
+ test
+
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java
new file mode 100644
index 0000000..ea0510f
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.InetSocketAddress;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.GetGroupsBase;
+import org.apache.hadoop.tools.GetUserMappingsProtocol;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.api.RMAdminProtocol;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+
+public class GetGroupsForTesting extends GetGroupsBase {
+
+ public GetGroupsForTesting(Configuration conf) {
+ super(conf);
+ }
+
+ public GetGroupsForTesting(Configuration conf, PrintStream out) {
+ super(conf, out);
+ }
+
+ @Override
+ protected InetSocketAddress getProtocolAddress(Configuration conf)
+ throws IOException {
+ return conf.getSocketAddr(YarnConfiguration.RM_ADMIN_ADDRESS,
+ YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
+ YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
+ }
+
+ @Override
+ public void setConf(Configuration conf) {
+ conf = new YarnConfiguration(conf);
+ super.setConf(conf);
+ }
+
+ @Override
+ protected GetUserMappingsProtocol getUgmProtocol() throws IOException {
+ Configuration conf = getConf();
+
+ final InetSocketAddress addr = conf.getSocketAddr(
+ YarnConfiguration.RM_ADMIN_ADDRESS,
+ YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
+ YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
+ final YarnRPC rpc = YarnRPC.create(conf);
+
+ RMAdminProtocol adminProtocol = (RMAdminProtocol) rpc.getProxy(
+ RMAdminProtocol.class, addr, getConf());
+
+ return adminProtocol;
+ }
+
+ public static void main(String[] argv) throws Exception {
+ int res = ToolRunner.run(new GetGroupsForTesting(new YarnConfiguration()), argv);
+ System.exit(res);
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java
new file mode 100644
index 0000000..d377cfb
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java
@@ -0,0 +1,536 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.AMRMProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+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;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.client.AMRMClient.StoredContainerRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.apache.hadoop.yarn.service.Service.STATE;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class TestAMRMClient {
+ static Configuration conf = null;
+ static MiniYARNCluster yarnCluster = null;
+ static YarnClientImpl yarnClient = null;
+ static List nodeReports = null;
+ static ApplicationAttemptId attemptId = null;
+ static int nodeCount = 3;
+
+ static Resource capability;
+ static Priority priority;
+ static String node;
+ static String rack;
+ static String[] nodes;
+ static String[] racks;
+
+ @BeforeClass
+ public static void setup() throws Exception {
+ // start minicluster
+ conf = new YarnConfiguration();
+ yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
+ yarnCluster.init(conf);
+ yarnCluster.start();
+
+ // start rm client
+ yarnClient = new YarnClientImpl();
+ yarnClient.init(conf);
+ yarnClient.start();
+
+ // get node info
+ nodeReports = yarnClient.getNodeReports();
+
+ priority = Priority.newInstance(1);
+ capability = Resource.newInstance(1024, 1);
+
+ node = nodeReports.get(0).getNodeId().getHost();
+ rack = nodeReports.get(0).getRackName();
+ nodes = new String[]{ node };
+ racks = new String[]{ rack };
+ }
+
+ @Before
+ public void startApp() throws Exception {
+ // submit new app
+ GetNewApplicationResponse newApp = yarnClient.getNewApplication();
+ ApplicationId appId = newApp.getApplicationId();
+
+ ApplicationSubmissionContext appContext = Records
+ .newRecord(ApplicationSubmissionContext.class);
+ // set the application id
+ appContext.setApplicationId(appId);
+ // set the application name
+ appContext.setApplicationName("Test");
+ // Set the priority for the application master
+ Priority pri = Records.newRecord(Priority.class);
+ pri.setPriority(0);
+ appContext.setPriority(pri);
+ // Set the queue to which this application is to be submitted in the RM
+ appContext.setQueue("default");
+ // Set up the container launch context for the application master
+ ContainerLaunchContext amContainer = Records
+ .newRecord(ContainerLaunchContext.class);
+ appContext.setAMContainerSpec(amContainer);
+ // unmanaged AM
+ appContext.setUnmanagedAM(true);
+ // Create the request to send to the applications manager
+ SubmitApplicationRequest appRequest = Records
+ .newRecord(SubmitApplicationRequest.class);
+ appRequest.setApplicationSubmissionContext(appContext);
+ // Submit the application to the applications manager
+ yarnClient.submitApplication(appContext);
+
+ // wait for app to start
+ while (true) {
+ ApplicationReport appReport = yarnClient.getApplicationReport(appId);
+ if (appReport.getYarnApplicationState() == YarnApplicationState.ACCEPTED) {
+ attemptId = appReport.getCurrentApplicationAttemptId();
+ break;
+ }
+ }
+ }
+
+ @After
+ public void cancelApp() {
+ attemptId = null;
+ }
+
+ @AfterClass
+ public static void tearDown() {
+ if (yarnClient != null && yarnClient.getServiceState() == STATE.STARTED) {
+ yarnClient.stop();
+ }
+ if (yarnCluster != null && yarnCluster.getServiceState() == STATE.STARTED) {
+ yarnCluster.stop();
+ }
+ }
+
+ @Test (timeout=60000)
+ public void testAMRMClientMatchingFit() throws YarnException, IOException {
+ AMRMClientImpl amClient = null;
+ try {
+ // start am rm client
+ amClient = new AMRMClientImpl(attemptId);
+ amClient.init(conf);
+ amClient.start();
+ amClient.registerApplicationMaster("Host", 10000, "");
+
+ Resource capability1 = Resource.newInstance(1024, 2);
+ Resource capability2 = Resource.newInstance(1024, 1);
+ Resource capability3 = Resource.newInstance(1000, 2);
+ Resource capability4 = Resource.newInstance(2000, 1);
+ Resource capability5 = Resource.newInstance(1000, 3);
+ Resource capability6 = Resource.newInstance(2000, 1);
+
+ StoredContainerRequest storedContainer1 =
+ new StoredContainerRequest(capability1, nodes, racks, priority);
+ StoredContainerRequest storedContainer2 =
+ new StoredContainerRequest(capability2, nodes, racks, priority);
+ StoredContainerRequest storedContainer3 =
+ new StoredContainerRequest(capability3, nodes, racks, priority);
+ StoredContainerRequest storedContainer4 =
+ new StoredContainerRequest(capability4, nodes, racks, priority);
+ StoredContainerRequest storedContainer5 =
+ new StoredContainerRequest(capability5, nodes, racks, priority);
+ StoredContainerRequest storedContainer6 =
+ new StoredContainerRequest(capability6, nodes, racks, priority);
+ amClient.addContainerRequest(storedContainer1);
+ amClient.addContainerRequest(storedContainer2);
+ amClient.addContainerRequest(storedContainer3);
+ amClient.addContainerRequest(storedContainer4);
+ amClient.addContainerRequest(storedContainer5);
+ amClient.addContainerRequest(storedContainer6);
+
+ // test matching of containers
+ List extends Collection> matches;
+ StoredContainerRequest storedRequest;
+ // exact match
+ Resource testCapability1 = Resource.newInstance(1024, 2);
+ matches = amClient.getMatchingRequests(priority, node, testCapability1);
+ verifyMatches(matches, 1);
+ storedRequest = matches.get(0).iterator().next();
+ assertTrue(storedContainer1 == storedRequest);
+ amClient.removeContainerRequest(storedContainer1);
+
+ // exact matching with order maintained
+ Resource testCapability2 = Resource.newInstance(2000, 1);
+ matches = amClient.getMatchingRequests(priority, node, testCapability2);
+ verifyMatches(matches, 2);
+ // must be returned in the order they were made
+ int i = 0;
+ for(StoredContainerRequest storedRequest1 : matches.get(0)) {
+ if(i++ == 0) {
+ assertTrue(storedContainer4 == storedRequest1);
+ } else {
+ assertTrue(storedContainer6 == storedRequest1);
+ }
+ }
+ amClient.removeContainerRequest(storedContainer6);
+
+ // matching with larger container. all requests returned
+ Resource testCapability3 = Resource.newInstance(4000, 4);
+ matches = amClient.getMatchingRequests(priority, node, testCapability3);
+ assert(matches.size() == 4);
+
+ Resource testCapability4 = Resource.newInstance(1024, 2);
+ matches = amClient.getMatchingRequests(priority, node, testCapability4);
+ assert(matches.size() == 2);
+ // verify non-fitting containers are not returned and fitting ones are
+ for(Collection testSet : matches) {
+ assertTrue(testSet.size() == 1);
+ StoredContainerRequest testRequest = testSet.iterator().next();
+ assertTrue(testRequest != storedContainer4);
+ assertTrue(testRequest != storedContainer5);
+ assert(testRequest == storedContainer2 ||
+ testRequest == storedContainer3);
+ }
+
+ Resource testCapability5 = Resource.newInstance(512, 4);
+ matches = amClient.getMatchingRequests(priority, node, testCapability5);
+ assert(matches.size() == 0);
+
+ amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
+ null, null);
+ } finally {
+ if (amClient != null && amClient.getServiceState() == STATE.STARTED) {
+ amClient.stop();
+ }
+ }
+ }
+
+ private void verifyMatches(
+ List extends Collection> matches,
+ int matchSize) {
+ assertTrue(matches.size() == 1);
+ assertTrue(matches.get(0).size() == matchSize);
+ }
+
+ @Test (timeout=60000)
+ public void testAMRMClientMatchStorage() throws YarnException, IOException {
+ AMRMClientImpl amClient = null;
+ try {
+ // start am rm client
+ amClient = new AMRMClientImpl(attemptId);
+ amClient.init(conf);
+ amClient.start();
+ amClient.registerApplicationMaster("Host", 10000, "");
+
+ Priority priority1 = Records.newRecord(Priority.class);
+ priority1.setPriority(2);
+
+ StoredContainerRequest storedContainer1 =
+ new StoredContainerRequest(capability, nodes, racks, priority);
+ StoredContainerRequest storedContainer2 =
+ new StoredContainerRequest(capability, nodes, racks, priority);
+ StoredContainerRequest storedContainer3 =
+ new StoredContainerRequest(capability, null, null, priority1);
+ amClient.addContainerRequest(storedContainer1);
+ amClient.addContainerRequest(storedContainer2);
+ amClient.addContainerRequest(storedContainer3);
+
+ // test addition and storage
+ int containersRequestedAny = amClient.remoteRequestsTable.get(priority)
+ .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
+ assertTrue(containersRequestedAny == 2);
+ containersRequestedAny = amClient.remoteRequestsTable.get(priority1)
+ .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
+ assertTrue(containersRequestedAny == 1);
+ List extends Collection> matches =
+ amClient.getMatchingRequests(priority, node, capability);
+ verifyMatches(matches, 2);
+ matches = amClient.getMatchingRequests(priority, rack, capability);
+ verifyMatches(matches, 2);
+ matches =
+ amClient.getMatchingRequests(priority, ResourceRequest.ANY, capability);
+ verifyMatches(matches, 2);
+ matches = amClient.getMatchingRequests(priority1, rack, capability);
+ assertTrue(matches.isEmpty());
+ matches =
+ amClient.getMatchingRequests(priority1, ResourceRequest.ANY, capability);
+ verifyMatches(matches, 1);
+
+ // test removal
+ amClient.removeContainerRequest(storedContainer3);
+ matches = amClient.getMatchingRequests(priority, node, capability);
+ verifyMatches(matches, 2);
+ amClient.removeContainerRequest(storedContainer2);
+ matches = amClient.getMatchingRequests(priority, node, capability);
+ verifyMatches(matches, 1);
+ matches = amClient.getMatchingRequests(priority, rack, capability);
+ verifyMatches(matches, 1);
+
+ // test matching of containers
+ StoredContainerRequest storedRequest = matches.get(0).iterator().next();
+ assertTrue(storedContainer1 == storedRequest);
+ amClient.removeContainerRequest(storedContainer1);
+ matches =
+ amClient.getMatchingRequests(priority, ResourceRequest.ANY, capability);
+ assertTrue(matches.isEmpty());
+ matches =
+ amClient.getMatchingRequests(priority1, ResourceRequest.ANY, capability);
+ assertTrue(matches.isEmpty());
+ // 0 requests left. everything got cleaned up
+ assertTrue(amClient.remoteRequestsTable.isEmpty());
+
+ // go through an exemplary allocation, matching and release cycle
+ amClient.addContainerRequest(storedContainer1);
+ amClient.addContainerRequest(storedContainer3);
+ // RM should allocate container within 2 calls to allocate()
+ int allocatedContainerCount = 0;
+ int iterationsLeft = 2;
+ while (allocatedContainerCount < 2
+ && iterationsLeft-- > 0) {
+ AllocateResponse allocResponse = amClient.allocate(0.1f);
+ assertTrue(amClient.ask.size() == 0);
+ assertTrue(amClient.release.size() == 0);
+
+ assertTrue(nodeCount == amClient.getClusterNodeCount());
+ allocatedContainerCount += allocResponse.getAllocatedContainers().size();
+ for(Container container : allocResponse.getAllocatedContainers()) {
+ ContainerRequest expectedRequest =
+ container.getPriority().equals(storedContainer1.getPriority()) ?
+ storedContainer1 : storedContainer3;
+ matches = amClient.getMatchingRequests(container.getPriority(),
+ ResourceRequest.ANY,
+ container.getResource());
+ // test correct matched container is returned
+ verifyMatches(matches, 1);
+ ContainerRequest matchedRequest = matches.get(0).iterator().next();
+ assertTrue(matchedRequest == expectedRequest);
+
+ // assign this container, use it and release it
+ amClient.releaseAssignedContainer(container.getId());
+ }
+ if(allocatedContainerCount < containersRequestedAny) {
+ // sleep to let NM's heartbeat to RM and trigger allocations
+ sleep(1000);
+ }
+ }
+
+ assertTrue(allocatedContainerCount == 2);
+ assertTrue(amClient.release.size() == 2);
+ assertTrue(amClient.ask.size() == 0);
+ AllocateResponse allocResponse = amClient.allocate(0.1f);
+ assertTrue(amClient.release.size() == 0);
+ assertTrue(amClient.ask.size() == 0);
+ assertTrue(allocResponse.getAllocatedContainers().size() == 0);
+
+ amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
+ null, null);
+ } finally {
+ if (amClient != null && amClient.getServiceState() == STATE.STARTED) {
+ amClient.stop();
+ }
+ }
+ }
+
+ @Test (timeout=60000)
+ public void testAMRMClient() throws YarnException, IOException {
+ AMRMClientImpl amClient = null;
+ try {
+ // start am rm client
+ amClient = new AMRMClientImpl(attemptId);
+ amClient.init(conf);
+ amClient.start();
+
+ amClient.registerApplicationMaster("Host", 10000, "");
+
+ testAllocation(amClient);
+
+ amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
+ null, null);
+
+ } finally {
+ if (amClient != null && amClient.getServiceState() == STATE.STARTED) {
+ amClient.stop();
+ }
+ }
+ }
+
+ private void testAllocation(final AMRMClientImpl amClient)
+ throws YarnException, IOException {
+ // setup container request
+
+ assertTrue(amClient.ask.size() == 0);
+ assertTrue(amClient.release.size() == 0);
+
+ amClient.addContainerRequest(
+ new ContainerRequest(capability, nodes, racks, priority, 1));
+ amClient.addContainerRequest(
+ new ContainerRequest(capability, nodes, racks, priority, 3));
+ amClient.removeContainerRequest(
+ new ContainerRequest(capability, nodes, racks, priority, 2));
+
+ int containersRequestedNode = amClient.remoteRequestsTable.get(priority)
+ .get(node).get(capability).remoteRequest.getNumContainers();
+ int containersRequestedRack = amClient.remoteRequestsTable.get(priority)
+ .get(rack).get(capability).remoteRequest.getNumContainers();
+ int containersRequestedAny = amClient.remoteRequestsTable.get(priority)
+ .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
+
+ assertTrue(containersRequestedNode == 2);
+ assertTrue(containersRequestedRack == 2);
+ assertTrue(containersRequestedAny == 2);
+ assertTrue(amClient.ask.size() == 3);
+ assertTrue(amClient.release.size() == 0);
+
+ // RM should allocate container within 2 calls to allocate()
+ int allocatedContainerCount = 0;
+ int iterationsLeft = 2;
+ Set releases = new TreeSet();
+ while (allocatedContainerCount < containersRequestedAny
+ && iterationsLeft-- > 0) {
+ AllocateResponse allocResponse = amClient.allocate(0.1f);
+ assertTrue(amClient.ask.size() == 0);
+ assertTrue(amClient.release.size() == 0);
+
+ assertTrue(nodeCount == amClient.getClusterNodeCount());
+ allocatedContainerCount += allocResponse.getAllocatedContainers().size();
+ for(Container container : allocResponse.getAllocatedContainers()) {
+ ContainerId rejectContainerId = container.getId();
+ releases.add(rejectContainerId);
+ amClient.releaseAssignedContainer(rejectContainerId);
+ }
+ if(allocatedContainerCount < containersRequestedAny) {
+ // sleep to let NM's heartbeat to RM and trigger allocations
+ sleep(1000);
+ }
+ }
+
+ assertTrue(allocatedContainerCount == containersRequestedAny);
+ assertTrue(amClient.release.size() == 2);
+ assertTrue(amClient.ask.size() == 0);
+
+ // need to tell the AMRMClient that we dont need these resources anymore
+ amClient.removeContainerRequest(
+ new ContainerRequest(capability, nodes, racks, priority, 2));
+ assertTrue(amClient.ask.size() == 3);
+ // send 0 container count request for resources that are no longer needed
+ ResourceRequest snoopRequest = amClient.ask.iterator().next();
+ assertTrue(snoopRequest.getNumContainers() == 0);
+
+ // test RPC exception handling
+ amClient.addContainerRequest(new ContainerRequest(capability, nodes,
+ racks, priority, 2));
+ snoopRequest = amClient.ask.iterator().next();
+ assertTrue(snoopRequest.getNumContainers() == 2);
+
+ AMRMProtocol realRM = amClient.rmClient;
+ try {
+ AMRMProtocol mockRM = mock(AMRMProtocol.class);
+ when(mockRM.allocate(any(AllocateRequest.class))).thenAnswer(
+ new Answer() {
+ public AllocateResponse answer(InvocationOnMock invocation)
+ throws Exception {
+ amClient.removeContainerRequest(
+ new ContainerRequest(capability, nodes,
+ racks, priority, 2));
+ throw new Exception();
+ }
+ });
+ amClient.rmClient = mockRM;
+ amClient.allocate(0.1f);
+ }catch (Exception ioe) {}
+ finally {
+ amClient.rmClient = realRM;
+ }
+
+ assertTrue(amClient.release.size() == 2);
+ assertTrue(amClient.ask.size() == 3);
+ snoopRequest = amClient.ask.iterator().next();
+ // verify that the remove request made in between makeRequest and allocate
+ // has not been lost
+ assertTrue(snoopRequest.getNumContainers() == 0);
+
+ iterationsLeft = 2;
+ // do a few iterations to ensure RM is not going send new containers
+ while(!releases.isEmpty() || iterationsLeft-- > 0) {
+ // inform RM of rejection
+ AllocateResponse allocResponse = amClient.allocate(0.1f);
+ // RM did not send new containers because AM does not need any
+ assertTrue(allocResponse.getAllocatedContainers().size() == 0);
+ if(allocResponse.getCompletedContainersStatuses().size() > 0) {
+ for(ContainerStatus cStatus :allocResponse
+ .getCompletedContainersStatuses()) {
+ if(releases.contains(cStatus.getContainerId())) {
+ assertTrue(cStatus.getState() == ContainerState.COMPLETE);
+ assertTrue(cStatus.getExitStatus() == -100);
+ releases.remove(cStatus.getContainerId());
+ }
+ }
+ }
+ if(iterationsLeft > 0) {
+ // sleep to make sure NM's heartbeat
+ sleep(1000);
+ }
+ }
+
+ assertTrue(amClient.ask.size() == 0);
+ assertTrue(amClient.release.size() == 0);
+ }
+
+ private void sleep(int sleepTime) {
+ try {
+ Thread.sleep(sleepTime);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java
new file mode 100644
index 0000000..ac8e3cd
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java
@@ -0,0 +1,320 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client;
+
+import static org.mockito.Matchers.anyFloat;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+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.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.AMCommand;
+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;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class TestAMRMClientAsync {
+
+ private static final Log LOG = LogFactory.getLog(TestAMRMClientAsync.class);
+
+ @SuppressWarnings("unchecked")
+ @Test(timeout=10000)
+ public void testAMRMClientAsync() throws Exception {
+ Configuration conf = new Configuration();
+ final AtomicBoolean heartbeatBlock = new AtomicBoolean(true);
+ List completed1 = Arrays.asList(
+ ContainerStatus.newInstance(newContainerId(0, 0, 0, 0),
+ ContainerState.COMPLETE, "", 0));
+ List allocated1 = Arrays.asList(
+ Container.newInstance(null, null, null, null, null, null));
+ final AllocateResponse response1 = createAllocateResponse(
+ new ArrayList(), allocated1);
+ final AllocateResponse response2 = createAllocateResponse(completed1,
+ new ArrayList());
+ final AllocateResponse emptyResponse = createAllocateResponse(
+ new ArrayList(), new ArrayList());
+
+ TestCallbackHandler callbackHandler = new TestCallbackHandler();
+ final AMRMClient client = mock(AMRMClientImpl.class);
+ final AtomicInteger secondHeartbeatSync = new AtomicInteger(0);
+ when(client.allocate(anyFloat())).thenReturn(response1).thenAnswer(new Answer() {
+ @Override
+ public AllocateResponse answer(InvocationOnMock invocation)
+ throws Throwable {
+ secondHeartbeatSync.incrementAndGet();
+ while(heartbeatBlock.get()) {
+ synchronized(heartbeatBlock) {
+ heartbeatBlock.wait();
+ }
+ }
+ secondHeartbeatSync.incrementAndGet();
+ return response2;
+ }
+ }).thenReturn(emptyResponse);
+ when(client.registerApplicationMaster(anyString(), anyInt(), anyString()))
+ .thenReturn(null);
+ when(client.getClusterAvailableResources()).thenAnswer(new Answer() {
+ @Override
+ public Resource answer(InvocationOnMock invocation)
+ throws Throwable {
+ // take client lock to simulate behavior of real impl
+ synchronized (client) {
+ Thread.sleep(10);
+ }
+ return null;
+ }
+ });
+
+ AMRMClientAsync asyncClient =
+ new AMRMClientAsync(client, 20, callbackHandler);
+ asyncClient.init(conf);
+ asyncClient.start();
+ asyncClient.registerApplicationMaster("localhost", 1234, null);
+
+ // while the CallbackHandler will still only be processing the first response,
+ // heartbeater thread should still be sending heartbeats.
+ // To test this, wait for the second heartbeat to be received.
+ while (secondHeartbeatSync.get() < 1) {
+ Thread.sleep(10);
+ }
+
+ // heartbeat will be blocked. make sure we can call client methods at this
+ // time. Checks that heartbeat is not holding onto client lock
+ assert(secondHeartbeatSync.get() < 2);
+ asyncClient.getClusterAvailableResources();
+ // method returned. now unblock heartbeat
+ assert(secondHeartbeatSync.get() < 2);
+ synchronized (heartbeatBlock) {
+ heartbeatBlock.set(false);
+ heartbeatBlock.notifyAll();
+ }
+
+ // allocated containers should come before completed containers
+ Assert.assertEquals(null, callbackHandler.takeCompletedContainers());
+
+ // wait for the allocated containers from the first heartbeat's response
+ while (callbackHandler.takeAllocatedContainers() == null) {
+ Assert.assertEquals(null, callbackHandler.takeCompletedContainers());
+ Thread.sleep(10);
+ }
+
+ // wait for the completed containers from the second heartbeat's response
+ while (callbackHandler.takeCompletedContainers() == null) {
+ Thread.sleep(10);
+ }
+
+ asyncClient.stop();
+
+ Assert.assertEquals(null, callbackHandler.takeAllocatedContainers());
+ Assert.assertEquals(null, callbackHandler.takeCompletedContainers());
+ }
+
+ @Test(timeout=10000)
+ public void testAMRMClientAsyncException() throws Exception {
+ Configuration conf = new Configuration();
+ TestCallbackHandler callbackHandler = new TestCallbackHandler();
+ @SuppressWarnings("unchecked")
+ AMRMClient client = mock(AMRMClientImpl.class);
+ String exStr = "TestException";
+ YarnException mockException = mock(YarnException.class);
+ when(mockException.getMessage()).thenReturn(exStr);
+ when(client.allocate(anyFloat())).thenThrow(mockException);
+
+ AMRMClientAsync asyncClient =
+ new AMRMClientAsync(client, 20, callbackHandler);
+ asyncClient.init(conf);
+ asyncClient.start();
+
+ synchronized (callbackHandler.notifier) {
+ asyncClient.registerApplicationMaster("localhost", 1234, null);
+ while(callbackHandler.savedException == null) {
+ try {
+ callbackHandler.notifier.wait();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ Assert.assertTrue(callbackHandler.savedException.getMessage().contains(exStr));
+
+ asyncClient.stop();
+ // stopping should have joined all threads and completed all callbacks
+ Assert.assertTrue(callbackHandler.callbackCount == 0);
+ }
+
+ @Test//(timeout=10000)
+ public void testAMRMClientAsyncReboot() throws Exception {
+ Configuration conf = new Configuration();
+ TestCallbackHandler callbackHandler = new TestCallbackHandler();
+ @SuppressWarnings("unchecked")
+ AMRMClient client = mock(AMRMClientImpl.class);
+
+ final AllocateResponse rebootResponse = createAllocateResponse(
+ new ArrayList(), new ArrayList());
+ rebootResponse.setAMCommand(AMCommand.AM_RESYNC);
+ when(client.allocate(anyFloat())).thenReturn(rebootResponse);
+
+ AMRMClientAsync asyncClient =
+ new AMRMClientAsync(client, 20, callbackHandler);
+ asyncClient.init(conf);
+ asyncClient.start();
+
+ synchronized (callbackHandler.notifier) {
+ asyncClient.registerApplicationMaster("localhost", 1234, null);
+ while(callbackHandler.reboot == false) {
+ try {
+ callbackHandler.notifier.wait();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ asyncClient.stop();
+ // stopping should have joined all threads and completed all callbacks
+ Assert.assertTrue(callbackHandler.callbackCount == 0);
+ }
+
+ private AllocateResponse createAllocateResponse(
+ List completed, List allocated) {
+ AllocateResponse response = AllocateResponse.newInstance(0, completed, allocated,
+ new ArrayList(), null, null, 1, null);
+ return response;
+ }
+
+ public static ContainerId newContainerId(int appId, int appAttemptId,
+ long timestamp, int containerId) {
+ ApplicationId applicationId = ApplicationId.newInstance(timestamp, appId);
+ ApplicationAttemptId applicationAttemptId =
+ ApplicationAttemptId.newInstance(applicationId, appAttemptId);
+ return ContainerId.newInstance(applicationAttemptId, containerId);
+ }
+
+ private class TestCallbackHandler implements AMRMClientAsync.CallbackHandler {
+ private volatile List completedContainers;
+ private volatile List allocatedContainers;
+ Exception savedException = null;
+ boolean reboot = false;
+ Object notifier = new Object();
+
+ int callbackCount = 0;
+
+ public List takeCompletedContainers() {
+ List ret = completedContainers;
+ if (ret == null) {
+ return null;
+ }
+ completedContainers = null;
+ synchronized (ret) {
+ ret.notify();
+ }
+ return ret;
+ }
+
+ public List takeAllocatedContainers() {
+ List ret = allocatedContainers;
+ if (ret == null) {
+ return null;
+ }
+ allocatedContainers = null;
+ synchronized (ret) {
+ ret.notify();
+ }
+ return ret;
+ }
+
+ @Override
+ public void onContainersCompleted(List statuses) {
+ completedContainers = statuses;
+ // wait for containers to be taken before returning
+ synchronized (completedContainers) {
+ while (completedContainers != null) {
+ try {
+ completedContainers.wait();
+ } catch (InterruptedException ex) {
+ LOG.error("Interrupted during wait", ex);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void onContainersAllocated(List containers) {
+ allocatedContainers = containers;
+ // wait for containers to be taken before returning
+ synchronized (allocatedContainers) {
+ while (allocatedContainers != null) {
+ try {
+ allocatedContainers.wait();
+ } catch (InterruptedException ex) {
+ LOG.error("Interrupted during wait", ex);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void onShutdownRequest() {
+ reboot = true;
+ synchronized (notifier) {
+ notifier.notifyAll();
+ }
+ }
+
+ @Override
+ public void onNodesUpdated(List updatedNodes) {}
+
+ @Override
+ public float getProgress() {
+ callbackCount++;
+ return 0.5f;
+ }
+
+ @Override
+ public void onError(Exception e) {
+ savedException = e;
+ synchronized (notifier) {
+ notifier.notifyAll();
+ }
+ }
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java
new file mode 100644
index 0000000..c0fcbe8
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.GetGroupsTestBase;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.service.Service.STATE;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+public class TestGetGroups extends GetGroupsTestBase {
+
+ private static final Log LOG = LogFactory.getLog(TestGetGroups.class);
+
+ private static ResourceManager resourceManager;
+
+ private static Configuration conf;
+
+ @BeforeClass
+ public static void setUpResourceManager() throws IOException, InterruptedException {
+ conf = new YarnConfiguration();
+ resourceManager = new ResourceManager() {
+ @Override
+ protected void doSecureLogin() throws IOException {
+ };
+ };
+ resourceManager.init(conf);
+ new Thread() {
+ public void run() {
+ resourceManager.start();
+ };
+ }.start();
+ int waitCount = 0;
+ while (resourceManager.getServiceState() == STATE.INITED
+ && waitCount++ < 10) {
+ LOG.info("Waiting for RM to start...");
+ Thread.sleep(1000);
+ }
+ if (resourceManager.getServiceState() != STATE.STARTED) {
+ throw new IOException(
+ "ResourceManager failed to start. Final state is "
+ + resourceManager.getServiceState());
+ }
+ LOG.info("ResourceManager RMAdmin address: " +
+ conf.get(YarnConfiguration.RM_ADMIN_ADDRESS));
+ }
+
+ @SuppressWarnings("static-access")
+ @Before
+ public void setUpConf() {
+ super.conf = this.conf;
+ }
+
+ @AfterClass
+ public static void tearDownResourceManager() throws InterruptedException {
+ if (resourceManager != null) {
+ LOG.info("Stopping ResourceManager...");
+ resourceManager.stop();
+ }
+ }
+
+ @Override
+ protected Tool getTool(PrintStream o) {
+ return new GetGroupsForTesting(conf, o);
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java
new file mode 100644
index 0000000..6f46ded
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java
@@ -0,0 +1,324 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.apache.hadoop.yarn.service.Service.STATE;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestNMClient {
+ Configuration conf = null;
+ MiniYARNCluster yarnCluster = null;
+ YarnClientImpl yarnClient = null;
+ AMRMClientImpl rmClient = null;
+ NMClientImpl nmClient = null;
+ List nodeReports = null;
+ ApplicationAttemptId attemptId = null;
+ int nodeCount = 3;
+
+ @Before
+ public void setup() throws YarnException, IOException {
+ // start minicluster
+ conf = new YarnConfiguration();
+ yarnCluster =
+ new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
+ yarnCluster.init(conf);
+ yarnCluster.start();
+ assertNotNull(yarnCluster);
+ assertEquals(STATE.STARTED, yarnCluster.getServiceState());
+
+ // start rm client
+ yarnClient = new YarnClientImpl();
+ yarnClient.init(conf);
+ yarnClient.start();
+ assertNotNull(yarnClient);
+ assertEquals(STATE.STARTED, yarnClient.getServiceState());
+
+ // get node info
+ nodeReports = yarnClient.getNodeReports();
+
+ // submit new app
+ GetNewApplicationResponse newApp = yarnClient.getNewApplication();
+ ApplicationId appId = newApp.getApplicationId();
+
+ ApplicationSubmissionContext appContext = Records
+ .newRecord(ApplicationSubmissionContext.class);
+ // set the application id
+ appContext.setApplicationId(appId);
+ // set the application name
+ appContext.setApplicationName("Test");
+ // Set the priority for the application master
+ Priority pri = Priority.newInstance(0);
+ appContext.setPriority(pri);
+ // Set the queue to which this application is to be submitted in the RM
+ appContext.setQueue("default");
+ // Set up the container launch context for the application master
+ ContainerLaunchContext amContainer = Records
+ .newRecord(ContainerLaunchContext.class);
+ appContext.setAMContainerSpec(amContainer);
+ // unmanaged AM
+ appContext.setUnmanagedAM(true);
+ // Create the request to send to the applications manager
+ SubmitApplicationRequest appRequest = Records
+ .newRecord(SubmitApplicationRequest.class);
+ appRequest.setApplicationSubmissionContext(appContext);
+ // Submit the application to the applications manager
+ yarnClient.submitApplication(appContext);
+
+ // wait for app to start
+ int iterationsLeft = 30;
+ while (iterationsLeft > 0) {
+ ApplicationReport appReport = yarnClient.getApplicationReport(appId);
+ if (appReport.getYarnApplicationState() ==
+ YarnApplicationState.ACCEPTED) {
+ attemptId = appReport.getCurrentApplicationAttemptId();
+ break;
+ }
+ sleep(1000);
+ --iterationsLeft;
+ }
+ if (iterationsLeft == 0) {
+ fail("Application hasn't bee started");
+ }
+
+ // start am rm client
+ rmClient = new AMRMClientImpl(attemptId);
+ rmClient.init(conf);
+ rmClient.start();
+ assertNotNull(rmClient);
+ assertEquals(STATE.STARTED, rmClient.getServiceState());
+
+ // start am nm client
+ nmClient = new NMClientImpl();
+ nmClient.init(conf);
+ nmClient.start();
+ assertNotNull(nmClient);
+ assertEquals(STATE.STARTED, nmClient.getServiceState());
+ }
+
+ @After
+ public void tearDown() {
+ rmClient.stop();
+
+ // leave one unclosed
+ assertEquals(1, nmClient.startedContainers.size());
+ // default true
+ assertTrue(nmClient.cleanupRunningContainers.get());
+ // don't stop the running containers
+ nmClient.cleanupRunningContainersOnStop(false);
+ assertFalse(nmClient.cleanupRunningContainers.get());
+ nmClient.stop();
+ assertTrue(nmClient.startedContainers.size() > 0);
+ // stop the running containers
+ nmClient.cleanupRunningContainersOnStop(true);
+ assertTrue(nmClient.cleanupRunningContainers.get());
+ nmClient.stop();
+ assertEquals(0, nmClient.startedContainers.size());
+
+ yarnClient.stop();
+ yarnCluster.stop();
+ }
+
+ @Test (timeout = 60000)
+ public void testNMClient()
+ throws YarnException, IOException {
+
+ rmClient.registerApplicationMaster("Host", 10000, "");
+
+ testContainerManagement(nmClient, allocateContainers(rmClient, 5));
+
+ rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
+ null, null);
+ }
+
+ private Set allocateContainers(
+ AMRMClientImpl rmClient, int num)
+ throws YarnException, IOException {
+ // setup container request
+ Resource capability = Resource.newInstance(1024, 0);
+ Priority priority = Priority.newInstance(0);
+ String node = nodeReports.get(0).getNodeId().getHost();
+ String rack = nodeReports.get(0).getRackName();
+ String[] nodes = new String[] {node};
+ String[] racks = new String[] {rack};
+
+ for (int i = 0; i < num; ++i) {
+ rmClient.addContainerRequest(new ContainerRequest(capability, nodes,
+ racks, priority, 1));
+ }
+
+ int containersRequestedAny = rmClient.remoteRequestsTable.get(priority)
+ .get(ResourceRequest.ANY).get(capability).remoteRequest
+ .getNumContainers();
+
+ // RM should allocate container within 2 calls to allocate()
+ int allocatedContainerCount = 0;
+ int iterationsLeft = 2;
+ Set containers = new TreeSet();
+ while (allocatedContainerCount < containersRequestedAny
+ && iterationsLeft > 0) {
+ AllocateResponse allocResponse = rmClient.allocate(0.1f);
+
+ allocatedContainerCount += allocResponse.getAllocatedContainers().size();
+ for(Container container : allocResponse.getAllocatedContainers()) {
+ containers.add(container);
+ }
+ if(allocatedContainerCount < containersRequestedAny) {
+ // sleep to let NM's heartbeat to RM and trigger allocations
+ sleep(1000);
+ }
+
+ --iterationsLeft;
+ }
+ return containers;
+ }
+
+ private void testContainerManagement(NMClientImpl nmClient,
+ Set containers) throws YarnException, IOException {
+ int size = containers.size();
+ int i = 0;
+ for (Container container : containers) {
+ // getContainerStatus shouldn't be called before startContainer,
+ // otherwise, NodeManager cannot find the container
+ try {
+ nmClient.getContainerStatus(container.getId(), container.getNodeId(),
+ container.getContainerToken());
+ fail("Exception is expected");
+ } catch (YarnException e) {
+ assertTrue("The thrown exception is not expected",
+ e.getMessage().contains("is not handled by this NodeManager"));
+ }
+
+ // stopContainer shouldn't be called before startContainer,
+ // otherwise, an exception will be thrown
+ try {
+ nmClient.stopContainer(container.getId(), container.getNodeId(),
+ container.getContainerToken());
+ fail("Exception is expected");
+ } catch (YarnException e) {
+ assertTrue("The thrown exception is not expected",
+ e.getMessage().contains(
+ "is either not started yet or already stopped"));
+ }
+
+ Credentials ts = new Credentials();
+ DataOutputBuffer dob = new DataOutputBuffer();
+ ts.writeTokenStorageToStream(dob);
+ ByteBuffer securityTokens =
+ ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+ ContainerLaunchContext clc =
+ Records.newRecord(ContainerLaunchContext.class);
+ clc.setTokens(securityTokens);
+ try {
+ nmClient.startContainer(container, clc);
+ } catch (YarnException e) {
+ fail("Exception is not expected");
+ }
+
+ // leave one container unclosed
+ if (++i < size) {
+ // NodeManager may still need some time to make the container started
+ testGetContainerStatus(container, i, ContainerState.RUNNING, "",
+ -1000);
+
+ try {
+ nmClient.stopContainer(container.getId(), container.getNodeId(),
+ container.getContainerToken());
+ } catch (YarnException e) {
+ fail("Exception is not expected");
+ }
+
+ // getContainerStatus can be called after stopContainer
+ testGetContainerStatus(container, i, ContainerState.COMPLETE,
+ "Container killed by the ApplicationMaster.", 143);
+ }
+ }
+ }
+
+ private void sleep(int sleepTime) {
+ try {
+ Thread.sleep(sleepTime);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+
+ private void testGetContainerStatus(Container container, int index,
+ ContainerState state, String diagnostics, int exitStatus)
+ throws YarnException, IOException {
+ while (true) {
+ try {
+ ContainerStatus status = nmClient.getContainerStatus(
+ container.getId(), container.getNodeId(),
+ container.getContainerToken());
+ // NodeManager may still need some time to get the stable
+ // container status
+ if (status.getState() == state) {
+ assertEquals(container.getId(), status.getContainerId());
+ assertTrue("" + index + ": " + status.getDiagnostics(),
+ status.getDiagnostics().contains(diagnostics));
+ assertEquals(exitStatus, status.getExitStatus());
+ break;
+ }
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java
new file mode 100644
index 0000000..6fff279
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java
@@ -0,0 +1,543 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+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;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.junit.Test;
+
+
+public class TestNMClientAsync {
+
+ private final RecordFactory recordFactory =
+ RecordFactoryProvider.getRecordFactory(null);
+
+ private NMClientAsync asyncClient;
+ private NodeId nodeId;
+ private Token containerToken;
+
+ @Test (timeout = 30000)
+ public void testNMClientAsync() throws Exception {
+ Configuration conf = new Configuration();
+ conf.setInt(YarnConfiguration.NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE, 10);
+
+ // Threads to run are more than the max size of the thread pool
+ int expectedSuccess = 40;
+ int expectedFailure = 40;
+
+ asyncClient = new MockNMClientAsync1(expectedSuccess, expectedFailure);
+ asyncClient.init(conf);
+ Assert.assertEquals("The max thread pool size is not correctly set",
+ 10, asyncClient.maxThreadPoolSize);
+ asyncClient.start();
+
+
+ for (int i = 0; i < expectedSuccess + expectedFailure; ++i) {
+ if (i == expectedSuccess) {
+ while (!((TestCallbackHandler1) asyncClient.callbackHandler)
+ .isAllSuccessCallsExecuted()) {
+ Thread.sleep(10);
+ }
+ asyncClient.client = mockNMClient(1);
+ }
+ Container container = mockContainer(i);
+ ContainerLaunchContext clc =
+ recordFactory.newRecordInstance(ContainerLaunchContext.class);
+ asyncClient.startContainer(container, clc);
+ }
+ while (!((TestCallbackHandler1) asyncClient.callbackHandler)
+ .isStartAndQueryFailureCallsExecuted()) {
+ Thread.sleep(10);
+ }
+ asyncClient.client = mockNMClient(2);
+ ((TestCallbackHandler1) asyncClient.callbackHandler).path = false;
+ for (int i = 0; i < expectedFailure; ++i) {
+ Container container = mockContainer(
+ expectedSuccess + expectedFailure + i);
+ ContainerLaunchContext clc =
+ recordFactory.newRecordInstance(ContainerLaunchContext.class);
+ asyncClient.startContainer(container, clc);
+ }
+ while (!((TestCallbackHandler1) asyncClient.callbackHandler)
+ .isStopFailureCallsExecuted()) {
+ Thread.sleep(10);
+ }
+ for (String errorMsg :
+ ((TestCallbackHandler1) asyncClient.callbackHandler).errorMsgs) {
+ System.out.println(errorMsg);
+ }
+ Assert.assertEquals("Error occurs in CallbackHandler", 0,
+ ((TestCallbackHandler1) asyncClient.callbackHandler).errorMsgs.size());
+ for (String errorMsg : ((MockNMClientAsync1) asyncClient).errorMsgs) {
+ System.out.println(errorMsg);
+ }
+ Assert.assertEquals("Error occurs in ContainerEventProcessor", 0,
+ ((MockNMClientAsync1) asyncClient).errorMsgs.size());
+ // When the callback functions are all executed, the event processor threads
+ // may still not terminate and the containers may still not removed.
+ while (asyncClient.containers.size() > 0) {
+ Thread.sleep(10);
+ }
+ asyncClient.stop();
+ Assert.assertFalse(
+ "The thread of Container Management Event Dispatcher is still alive",
+ asyncClient.eventDispatcherThread.isAlive());
+ Assert.assertTrue("The thread pool is not shut down",
+ asyncClient.threadPool.isShutdown());
+ }
+
+ private class MockNMClientAsync1 extends NMClientAsync {
+ private Set errorMsgs =
+ Collections.synchronizedSet(new HashSet());
+
+ protected MockNMClientAsync1(int expectedSuccess, int expectedFailure)
+ throws YarnException, IOException {
+ super(MockNMClientAsync1.class.getName(), mockNMClient(0),
+ new TestCallbackHandler1(expectedSuccess, expectedFailure));
+ }
+
+ private class MockContainerEventProcessor extends ContainerEventProcessor {
+ public MockContainerEventProcessor(ContainerEvent event) {
+ super(event);
+ }
+
+ @Override
+ public void run() {
+ try {
+ super.run();
+ } catch (RuntimeException e) {
+ // If the unexpected throwable comes from error callback functions, it
+ // will break ContainerEventProcessor.run(). Therefore, monitor
+ // the exception here
+ errorMsgs.add("Unexpected throwable from callback functions should" +
+ " be ignored by Container " + event.getContainerId());
+ }
+ }
+ }
+
+ @Override
+ protected ContainerEventProcessor getContainerEventProcessor(
+ ContainerEvent event) {
+ return new MockContainerEventProcessor(event);
+ }
+ }
+
+ private class TestCallbackHandler1
+ implements NMClientAsync.CallbackHandler {
+
+ private boolean path = true;
+
+ private int expectedSuccess;
+ private int expectedFailure;
+
+ private AtomicInteger actualStartSuccess = new AtomicInteger(0);
+ private AtomicInteger actualStartFailure = new AtomicInteger(0);
+ private AtomicInteger actualQuerySuccess = new AtomicInteger(0);
+ private AtomicInteger actualQueryFailure = new AtomicInteger(0);
+ private AtomicInteger actualStopSuccess = new AtomicInteger(0);
+ private AtomicInteger actualStopFailure = new AtomicInteger(0);
+
+ private AtomicIntegerArray actualStartSuccessArray;
+ private AtomicIntegerArray actualStartFailureArray;
+ private AtomicIntegerArray actualQuerySuccessArray;
+ private AtomicIntegerArray actualQueryFailureArray;
+ private AtomicIntegerArray actualStopSuccessArray;
+ private AtomicIntegerArray actualStopFailureArray;
+
+ private Set errorMsgs =
+ Collections.synchronizedSet(new HashSet());
+
+ public TestCallbackHandler1(int expectedSuccess, int expectedFailure) {
+ this.expectedSuccess = expectedSuccess;
+ this.expectedFailure = expectedFailure;
+
+ actualStartSuccessArray = new AtomicIntegerArray(expectedSuccess);
+ actualStartFailureArray = new AtomicIntegerArray(expectedFailure);
+ actualQuerySuccessArray = new AtomicIntegerArray(expectedSuccess);
+ actualQueryFailureArray = new AtomicIntegerArray(expectedFailure);
+ actualStopSuccessArray = new AtomicIntegerArray(expectedSuccess);
+ actualStopFailureArray = new AtomicIntegerArray(expectedFailure);
+ }
+
+ @Override
+ public void onContainerStarted(ContainerId containerId,
+ Map allServiceResponse) {
+ if (path) {
+ if (containerId.getId() >= expectedSuccess) {
+ errorMsgs.add("Container " + containerId +
+ " should throw the exception onContainerStarted");
+ return;
+ }
+ actualStartSuccess.addAndGet(1);
+ actualStartSuccessArray.set(containerId.getId(), 1);
+
+ // move on to the following success tests
+ asyncClient.getContainerStatus(containerId, nodeId, containerToken);
+ } else {
+ // move on to the following failure tests
+ asyncClient.stopContainer(containerId, nodeId, containerToken);
+ }
+
+ // Shouldn't crash the test thread
+ throw new RuntimeException("Ignorable Exception");
+ }
+
+ @Override
+ public void onContainerStatusReceived(ContainerId containerId,
+ ContainerStatus containerStatus) {
+ if (containerId.getId() >= expectedSuccess) {
+ errorMsgs.add("Container " + containerId +
+ " should throw the exception onContainerStatusReceived");
+ return;
+ }
+ actualQuerySuccess.addAndGet(1);
+ actualQuerySuccessArray.set(containerId.getId(), 1);
+ // move on to the following success tests
+ asyncClient.stopContainer(containerId, nodeId, containerToken);
+
+ // Shouldn't crash the test thread
+ throw new RuntimeException("Ignorable Exception");
+ }
+
+ @Override
+ public void onContainerStopped(ContainerId containerId) {
+ if (containerId.getId() >= expectedSuccess) {
+ errorMsgs.add("Container " + containerId +
+ " should throw the exception onContainerStopped");
+ return;
+ }
+ actualStopSuccess.addAndGet(1);
+ actualStopSuccessArray.set(containerId.getId(), 1);
+
+ // Shouldn't crash the test thread
+ throw new RuntimeException("Ignorable Exception");
+ }
+
+ @Override
+ public void onStartContainerError(ContainerId containerId, Throwable t) {
+ // If the unexpected throwable comes from success callback functions, it
+ // will be handled by the error callback functions. Therefore, monitor
+ // the exception here
+ if (t instanceof RuntimeException) {
+ errorMsgs.add("Unexpected throwable from callback functions should be" +
+ " ignored by Container " + containerId);
+ }
+ if (containerId.getId() < expectedSuccess) {
+ errorMsgs.add("Container " + containerId +
+ " shouldn't throw the exception onStartContainerError");
+ return;
+ }
+ actualStartFailure.addAndGet(1);
+ actualStartFailureArray.set(containerId.getId() - expectedSuccess, 1);
+ // move on to the following failure tests
+ asyncClient.getContainerStatus(containerId, nodeId, containerToken);
+
+ // Shouldn't crash the test thread
+ throw new RuntimeException("Ignorable Exception");
+ }
+
+ @Override
+ public void onStopContainerError(ContainerId containerId, Throwable t) {
+ if (t instanceof RuntimeException) {
+ errorMsgs.add("Unexpected throwable from callback functions should be" +
+ " ignored by Container " + containerId);
+ }
+ if (containerId.getId() < expectedSuccess + expectedFailure) {
+ errorMsgs.add("Container " + containerId +
+ " shouldn't throw the exception onStopContainerError");
+ return;
+ }
+
+ actualStopFailure.addAndGet(1);
+ actualStopFailureArray.set(
+ containerId.getId() - expectedSuccess - expectedFailure, 1);
+
+ // Shouldn't crash the test thread
+ throw new RuntimeException("Ignorable Exception");
+ }
+
+ @Override
+ public void onGetContainerStatusError(ContainerId containerId,
+ Throwable t) {
+ if (t instanceof RuntimeException) {
+ errorMsgs.add("Unexpected throwable from callback functions should be"
+ + " ignored by Container " + containerId);
+ }
+ if (containerId.getId() < expectedSuccess) {
+ errorMsgs.add("Container " + containerId +
+ " shouldn't throw the exception onGetContainerStatusError");
+ return;
+ }
+ actualQueryFailure.addAndGet(1);
+ actualQueryFailureArray.set(containerId.getId() - expectedSuccess, 1);
+
+ // Shouldn't crash the test thread
+ throw new RuntimeException("Ignorable Exception");
+ }
+
+ public boolean isAllSuccessCallsExecuted() {
+ boolean isAllSuccessCallsExecuted =
+ actualStartSuccess.get() == expectedSuccess &&
+ actualQuerySuccess.get() == expectedSuccess &&
+ actualStopSuccess.get() == expectedSuccess;
+ if (isAllSuccessCallsExecuted) {
+ assertAtomicIntegerArray(actualStartSuccessArray);
+ assertAtomicIntegerArray(actualQuerySuccessArray);
+ assertAtomicIntegerArray(actualStopSuccessArray);
+ }
+ return isAllSuccessCallsExecuted;
+ }
+
+ public boolean isStartAndQueryFailureCallsExecuted() {
+ boolean isStartAndQueryFailureCallsExecuted =
+ actualStartFailure.get() == expectedFailure &&
+ actualQueryFailure.get() == expectedFailure;
+ if (isStartAndQueryFailureCallsExecuted) {
+ assertAtomicIntegerArray(actualStartFailureArray);
+ assertAtomicIntegerArray(actualQueryFailureArray);
+ }
+ return isStartAndQueryFailureCallsExecuted;
+ }
+
+ public boolean isStopFailureCallsExecuted() {
+ boolean isStopFailureCallsExecuted =
+ actualStopFailure.get() == expectedFailure;
+ if (isStopFailureCallsExecuted) {
+ assertAtomicIntegerArray(actualStopFailureArray);
+ }
+ return isStopFailureCallsExecuted;
+ }
+
+ private void assertAtomicIntegerArray(AtomicIntegerArray array) {
+ for (int i = 0; i < array.length(); ++i) {
+ Assert.assertEquals(1, array.get(i));
+ }
+ }
+ }
+
+ private NMClient mockNMClient(int mode)
+ throws YarnException, IOException {
+ NMClient client = mock(NMClient.class);
+ switch (mode) {
+ case 0:
+ when(client.startContainer(any(Container.class),
+ any(ContainerLaunchContext.class))).thenReturn(
+ Collections.emptyMap());
+ when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class),
+ any(Token.class))).thenReturn(
+ recordFactory.newRecordInstance(ContainerStatus.class));
+ doNothing().when(client).stopContainer(any(ContainerId.class),
+ any(NodeId.class), any(Token.class));
+ break;
+ case 1:
+ doThrow(RPCUtil.getRemoteException("Start Exception")).when(client)
+ .startContainer(any(Container.class),
+ any(ContainerLaunchContext.class));
+ doThrow(RPCUtil.getRemoteException("Query Exception")).when(client)
+ .getContainerStatus(any(ContainerId.class), any(NodeId.class),
+ any(Token.class));
+ doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client)
+ .stopContainer(any(ContainerId.class), any(NodeId.class),
+ any(Token.class));
+ break;
+ case 2:
+ when(client.startContainer(any(Container.class),
+ any(ContainerLaunchContext.class))).thenReturn(
+ Collections.emptyMap());
+ when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class),
+ any(Token.class))).thenReturn(
+ recordFactory.newRecordInstance(ContainerStatus.class));
+ doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client)
+ .stopContainer(any(ContainerId.class), any(NodeId.class),
+ any(Token.class));
+ }
+ return client;
+ }
+
+ @Test (timeout = 10000)
+ public void testOutOfOrder() throws Exception {
+ CyclicBarrier barrierA = new CyclicBarrier(2);
+ CyclicBarrier barrierB = new CyclicBarrier(2);
+ CyclicBarrier barrierC = new CyclicBarrier(2);
+ asyncClient = new MockNMClientAsync2(barrierA, barrierB, barrierC);
+ asyncClient.init(new Configuration());
+ asyncClient.start();
+
+ final Container container = mockContainer(1);
+ final ContainerLaunchContext clc =
+ recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+ // start container from another thread
+ Thread t = new Thread() {
+ @Override
+ public void run() {
+ asyncClient.startContainer(container, clc);
+ }
+ };
+ t.start();
+
+ barrierA.await();
+ asyncClient.stopContainer(container.getId(), container.getNodeId(),
+ container.getContainerToken());
+ barrierC.await();
+
+ Assert.assertFalse("Starting and stopping should be out of order",
+ ((TestCallbackHandler2) asyncClient.callbackHandler)
+ .exceptionOccurred.get());
+ }
+
+ private class MockNMClientAsync2 extends NMClientAsync {
+ private CyclicBarrier barrierA;
+ private CyclicBarrier barrierB;
+
+ protected MockNMClientAsync2(CyclicBarrier barrierA, CyclicBarrier barrierB,
+ CyclicBarrier barrierC) throws YarnException, IOException {
+ super(MockNMClientAsync2.class.getName(), mockNMClient(0),
+ new TestCallbackHandler2(barrierC));
+ this.barrierA = barrierA;
+ this.barrierB = barrierB;
+ }
+
+ private class MockContainerEventProcessor extends ContainerEventProcessor {
+
+ public MockContainerEventProcessor(ContainerEvent event) {
+ super(event);
+ }
+
+ @Override
+ public void run() {
+ try {
+ if (event.getType() == ContainerEventType.START_CONTAINER) {
+ barrierA.await();
+ barrierB.await();
+ }
+ super.run();
+ if (event.getType() == ContainerEventType.STOP_CONTAINER) {
+ barrierB.await();
+ }
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ } catch (BrokenBarrierException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ @Override
+ protected ContainerEventProcessor getContainerEventProcessor(
+ ContainerEvent event) {
+ return new MockContainerEventProcessor(event);
+ }
+ }
+
+ private class TestCallbackHandler2
+ implements NMClientAsync.CallbackHandler {
+ private CyclicBarrier barrierC;
+ private AtomicBoolean exceptionOccurred = new AtomicBoolean(false);
+
+ public TestCallbackHandler2(CyclicBarrier barrierC) {
+ this.barrierC = barrierC;
+ }
+
+ @Override
+ public void onContainerStarted(ContainerId containerId,
+ Map allServiceResponse) {
+ }
+
+ @Override
+ public void onContainerStatusReceived(ContainerId containerId,
+ ContainerStatus containerStatus) {
+ }
+
+ @Override
+ public void onContainerStopped(ContainerId containerId) {
+ }
+
+ @Override
+ public void onStartContainerError(ContainerId containerId, Throwable t) {
+ if (!t.getMessage().equals(NMClientAsync.StatefulContainer
+ .OutOfOrderTransition.STOP_BEFORE_START_ERROR_MSG)) {
+ exceptionOccurred.set(true);
+ return;
+ }
+ try {
+ barrierC.await();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ } catch (BrokenBarrierException e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Override
+ public void onGetContainerStatusError(ContainerId containerId,
+ Throwable t) {
+ }
+
+ @Override
+ public void onStopContainerError(ContainerId containerId, Throwable t) {
+ }
+
+ }
+
+ private Container mockContainer(int i) {
+ ApplicationId appId =
+ ApplicationId.newInstance(System.currentTimeMillis(), 1);
+ ApplicationAttemptId attemptId =
+ ApplicationAttemptId.newInstance(appId, 1);
+ ContainerId containerId = ContainerId.newInstance(attemptId, i);
+ nodeId = NodeId.newInstance("localhost", 0);
+ // Create an empty record
+ containerToken = recordFactory.newRecordInstance(Token.class);
+ return Container.newInstance(containerId, nodeId, null, null, null,
+ containerToken);
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java
new file mode 100644
index 0000000..016fc42
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java
@@ -0,0 +1,175 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.client;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.ClientRMProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.junit.Test;
+
+public class TestYarnClient {
+
+ @Test
+ public void test() {
+ // More to come later.
+ }
+
+ @Test
+ public void testClientStop() {
+ Configuration conf = new Configuration();
+ ResourceManager rm = new ResourceManager();
+ rm.init(conf);
+ rm.start();
+
+ YarnClient client = new YarnClientImpl();
+ client.init(conf);
+ client.start();
+ client.stop();
+ }
+
+ @Test (timeout = 30000)
+ public void testSubmitApplication() {
+ Configuration conf = new Configuration();
+ conf.setLong(YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS,
+ 100); // speed up tests
+ final YarnClient client = new MockYarnClient();
+ client.init(conf);
+ client.start();
+
+ YarnApplicationState[] exitStates = new YarnApplicationState[]
+ {
+ YarnApplicationState.SUBMITTED,
+ YarnApplicationState.ACCEPTED,
+ YarnApplicationState.RUNNING,
+ YarnApplicationState.FINISHED,
+ YarnApplicationState.FAILED,
+ YarnApplicationState.KILLED
+ };
+ for (int i = 0; i < exitStates.length; ++i) {
+ ApplicationSubmissionContext context =
+ mock(ApplicationSubmissionContext.class);
+ ApplicationId applicationId = ApplicationId.newInstance(
+ System.currentTimeMillis(), i);
+ when(context.getApplicationId()).thenReturn(applicationId);
+ ((MockYarnClient) client).setYarnApplicationState(exitStates[i]);
+ try {
+ client.submitApplication(context);
+ } catch (YarnException e) {
+ Assert.fail("Exception is not expected.");
+ } catch (IOException e) {
+ Assert.fail("Exception is not expected.");
+ }
+ verify(((MockYarnClient) client).mockReport,times(4 * i + 4))
+ .getYarnApplicationState();
+ }
+
+ client.stop();
+ }
+
+ @Test(timeout = 30000)
+ public void testApplicationType() throws Exception {
+ Logger rootLogger = LogManager.getRootLogger();
+ rootLogger.setLevel(Level.DEBUG);
+ MockRM rm = new MockRM();
+ rm.start();
+ RMApp app = rm.submitApp(2000);
+ RMApp app1 =
+ rm.submitApp(200, "name", "user",
+ new HashMap(), false, "default", -1,
+ null, "MAPREDUCE");
+ Assert.assertEquals("YARN", app.getApplicationType());
+ Assert.assertEquals("MAPREDUCE", app1.getApplicationType());
+ rm.stop();
+ }
+
+ @Test(timeout = 30000)
+ public void testApplicationTypeLimit() throws Exception {
+ Logger rootLogger = LogManager.getRootLogger();
+ rootLogger.setLevel(Level.DEBUG);
+ MockRM rm = new MockRM();
+ rm.start();
+ RMApp app1 =
+ rm.submitApp(200, "name", "user",
+ new HashMap(), false, "default", -1,
+ null, "MAPREDUCE-LENGTH-IS-20");
+ Assert.assertEquals("MAPREDUCE-LENGTH-IS-", app1.getApplicationType());
+ rm.stop();
+ }
+
+ private static class MockYarnClient extends YarnClientImpl {
+ private ApplicationReport mockReport;
+
+ public MockYarnClient() {
+ super();
+ }
+
+ @Override
+ public void start() {
+ rmClient = mock(ClientRMProtocol.class);
+ GetApplicationReportResponse mockResponse =
+ mock(GetApplicationReportResponse.class);
+ mockReport = mock(ApplicationReport.class);
+ try{
+ when(rmClient.getApplicationReport(any(
+ GetApplicationReportRequest.class))).thenReturn(mockResponse);
+ } catch (YarnException e) {
+ Assert.fail("Exception is not expected.");
+ } catch (IOException e) {
+ Assert.fail("Exception is not expected.");
+ }
+ when(mockResponse.getApplicationReport()).thenReturn(mockReport);
+ }
+
+ @Override
+ public void stop() {
+ }
+
+ public void setYarnApplicationState(YarnApplicationState state) {
+ when(mockReport.getYarnApplicationState()).thenReturn(
+ YarnApplicationState.NEW, YarnApplicationState.NEW_SAVING,
+ YarnApplicationState.NEW_SAVING, state);
+ }
+ }
+
+}