|
@@ -43,7 +43,6 @@ import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
import java.util.TreeSet;
|
|
|
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.io.DataOutputBuffer;
|
|
|
import org.apache.hadoop.io.Text;
|
|
@@ -56,24 +55,18 @@ import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
|
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.*;
|
|
|
-import org.apache.hadoop.yarn.client.ClientRMProxy;
|
|
|
import org.apache.hadoop.yarn.client.api.AMRMClient;
|
|
|
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
|
|
|
import org.apache.hadoop.yarn.client.api.InvalidContainerRequestException;
|
|
|
import org.apache.hadoop.yarn.client.api.NMClient;
|
|
|
import org.apache.hadoop.yarn.client.api.NMTokenCache;
|
|
|
-import org.apache.hadoop.yarn.client.api.YarnClient;
|
|
|
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.security.AMRMTokenIdentifier;
|
|
|
-import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
|
@@ -81,10 +74,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedule
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
|
-import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Assume;
|
|
|
-import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
import org.junit.runner.RunWith;
|
|
|
import org.junit.runners.Parameterized;
|
|
@@ -97,26 +88,8 @@ import org.eclipse.jetty.util.log.Log;
|
|
|
* Test application master client class to resource manager.
|
|
|
*/
|
|
|
@RunWith(value = Parameterized.class)
|
|
|
-public class TestAMRMClient {
|
|
|
- private String schedulerName = null;
|
|
|
- private boolean autoUpdate = false;
|
|
|
- private Configuration conf = null;
|
|
|
- private MiniYARNCluster yarnCluster = null;
|
|
|
- private YarnClient yarnClient = null;
|
|
|
- private List<NodeReport> nodeReports = null;
|
|
|
- private ApplicationAttemptId attemptId = null;
|
|
|
- private int nodeCount = 3;
|
|
|
-
|
|
|
- static final int rolling_interval_sec = 13;
|
|
|
- static final long am_expire_ms = 4000;
|
|
|
-
|
|
|
- private Resource capability;
|
|
|
- private Priority priority;
|
|
|
- private Priority priority2;
|
|
|
- private String node;
|
|
|
- private String rack;
|
|
|
- private String[] nodes;
|
|
|
- private String[] racks;
|
|
|
+public class TestAMRMClient extends BaseAMRMClientTest{
|
|
|
+
|
|
|
private final static int DEFAULT_ITERATION = 3;
|
|
|
|
|
|
public TestAMRMClient(String schedulerName, boolean autoUpdate) {
|
|
@@ -134,127 +107,6 @@ public class TestAMRMClient {
|
|
|
});
|
|
|
}
|
|
|
|
|
|
- @Before
|
|
|
- public void setup() throws Exception {
|
|
|
- conf = new YarnConfiguration();
|
|
|
- createClusterAndStartApplication(conf);
|
|
|
- }
|
|
|
-
|
|
|
- private void createClusterAndStartApplication(Configuration conf)
|
|
|
- throws Exception {
|
|
|
- // start minicluster
|
|
|
- this.conf = conf;
|
|
|
- if (autoUpdate) {
|
|
|
- conf.setBoolean(YarnConfiguration.RM_AUTO_UPDATE_CONTAINERS, true);
|
|
|
- }
|
|
|
- conf.set(YarnConfiguration.RM_SCHEDULER, schedulerName);
|
|
|
- conf.setLong(
|
|
|
- YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS,
|
|
|
- rolling_interval_sec);
|
|
|
- conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, am_expire_ms);
|
|
|
- conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 100);
|
|
|
- // set the minimum allocation so that resource decrease can go under 1024
|
|
|
- conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512);
|
|
|
- conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
|
|
|
- conf.setBoolean(
|
|
|
- YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
|
|
|
- conf.setInt(
|
|
|
- YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, 10);
|
|
|
- yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
|
|
|
- yarnCluster.init(conf);
|
|
|
- yarnCluster.start();
|
|
|
-
|
|
|
- // start rm client
|
|
|
- yarnClient = YarnClient.createYarnClient();
|
|
|
- yarnClient.init(conf);
|
|
|
- yarnClient.start();
|
|
|
-
|
|
|
- // get node info
|
|
|
- assertTrue("All node managers did not connect to the RM within the "
|
|
|
- + "allotted 5-second timeout",
|
|
|
- yarnCluster.waitForNodeManagersToConnect(5000L));
|
|
|
- nodeReports = yarnClient.getNodeReports(NodeState.RUNNING);
|
|
|
- assertEquals("Not all node managers were reported running",
|
|
|
- nodeCount, nodeReports.size());
|
|
|
-
|
|
|
- priority = Priority.newInstance(1);
|
|
|
- priority2 = Priority.newInstance(2);
|
|
|
- capability = Resource.newInstance(1024, 1);
|
|
|
-
|
|
|
- node = nodeReports.get(0).getNodeId().getHost();
|
|
|
- rack = nodeReports.get(0).getRackName();
|
|
|
- nodes = new String[]{ node };
|
|
|
- racks = new String[]{ rack };
|
|
|
-
|
|
|
- // submit new app
|
|
|
- ApplicationSubmissionContext appContext =
|
|
|
- yarnClient.createApplication().getApplicationSubmissionContext();
|
|
|
- ApplicationId appId = appContext.getApplicationId();
|
|
|
- // 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 =
|
|
|
- BuilderUtils.newContainerLaunchContext(
|
|
|
- Collections.<String, LocalResource> emptyMap(),
|
|
|
- new HashMap<String, String>(), Arrays.asList("sleep", "100"),
|
|
|
- new HashMap<String, ByteBuffer>(), null,
|
|
|
- new HashMap<ApplicationAccessType, String>());
|
|
|
- appContext.setAMContainerSpec(amContainer);
|
|
|
- appContext.setResource(Resource.newInstance(1024, 1));
|
|
|
- // 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
|
|
|
- RMAppAttempt appAttempt = null;
|
|
|
- while (true) {
|
|
|
- ApplicationReport appReport = yarnClient.getApplicationReport(appId);
|
|
|
- if (appReport.getYarnApplicationState() == YarnApplicationState.ACCEPTED) {
|
|
|
- attemptId = appReport.getCurrentApplicationAttemptId();
|
|
|
- appAttempt =
|
|
|
- yarnCluster.getResourceManager().getRMContext().getRMApps()
|
|
|
- .get(attemptId.getApplicationId()).getCurrentAppAttempt();
|
|
|
- while (true) {
|
|
|
- if (appAttempt.getAppAttemptState() == RMAppAttemptState.LAUNCHED) {
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- // Just dig into the ResourceManager and get the AMRMToken just for the sake
|
|
|
- // of testing.
|
|
|
- UserGroupInformation.setLoginUser(UserGroupInformation
|
|
|
- .createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()));
|
|
|
-
|
|
|
- // emulate RM setup of AMRM token in credentials by adding the token
|
|
|
- // *before* setting the token service
|
|
|
- UserGroupInformation.getCurrentUser().addToken(appAttempt.getAMRMToken());
|
|
|
- appAttempt.getAMRMToken().setService(ClientRMProxy.getAMRMTokenService(conf));
|
|
|
- }
|
|
|
-
|
|
|
- @After
|
|
|
- public void teardown() throws YarnException, IOException {
|
|
|
- yarnClient.killApplication(attemptId.getApplicationId());
|
|
|
- attemptId = null;
|
|
|
-
|
|
|
- if (yarnClient != null && yarnClient.getServiceState() == STATE.STARTED) {
|
|
|
- yarnClient.stop();
|
|
|
- }
|
|
|
- if (yarnCluster != null && yarnCluster.getServiceState() == STATE.STARTED) {
|
|
|
- yarnCluster.stop();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
@Test (timeout = 60000)
|
|
|
public void testAMRMClientNoMatchingRequests()
|
|
|
throws IOException, YarnException {
|
|
@@ -905,7 +757,7 @@ public class TestAMRMClient {
|
|
|
initAMRMClientAndTest(false);
|
|
|
}
|
|
|
|
|
|
- private void initAMRMClientAndTest(boolean useAllocReqId)
|
|
|
+ protected void initAMRMClientAndTest(boolean useAllocReqId)
|
|
|
throws YarnException, IOException {
|
|
|
AMRMClient<ContainerRequest> amClient = null;
|
|
|
try {
|
|
@@ -1946,7 +1798,7 @@ public class TestAMRMClient {
|
|
|
// Wait for enough time and make sure the roll_over happens
|
|
|
// At mean time, the old AMRMToken should continue to work
|
|
|
while (System.currentTimeMillis() - startTime <
|
|
|
- rolling_interval_sec * 1000) {
|
|
|
+ rollingIntervalSec * 1000) {
|
|
|
amClient.allocate(0.1f);
|
|
|
sleep(1000);
|
|
|
}
|