|
@@ -39,6 +39,7 @@ import java.util.Arrays;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
|
|
+import com.google.common.base.Supplier;
|
|
|
import org.apache.commons.cli.MissingArgumentException;
|
|
|
import org.apache.commons.io.FileUtils;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -53,6 +54,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.net.ServerSocketUtil;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.JarFinder;
|
|
|
import org.apache.hadoop.util.Shell;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
@@ -63,6 +65,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerReport;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
+import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
@@ -174,6 +177,8 @@ public class TestDistributedShell {
|
|
|
true);
|
|
|
conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
|
|
|
true);
|
|
|
+ conf.setBoolean(
|
|
|
+ YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
|
|
|
|
|
|
// ATS version specific settings
|
|
|
if (timelineVersion == 1.0f) {
|
|
@@ -1469,6 +1474,97 @@ public class TestDistributedShell {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ @TimelineVersion(2.0f)
|
|
|
+ public void testDSShellWithEnforceExecutionType() throws Exception {
|
|
|
+ Client client = new Client(new Configuration(yarnCluster.getConfig()));
|
|
|
+ try {
|
|
|
+ String[] args = {
|
|
|
+ "--jar",
|
|
|
+ APPMASTER_JAR,
|
|
|
+ "--num_containers",
|
|
|
+ "2",
|
|
|
+ "--master_memory",
|
|
|
+ "512",
|
|
|
+ "--master_vcores",
|
|
|
+ "2",
|
|
|
+ "--container_memory",
|
|
|
+ "128",
|
|
|
+ "--container_vcores",
|
|
|
+ "1",
|
|
|
+ "--shell_command",
|
|
|
+ "date",
|
|
|
+ "--container_type",
|
|
|
+ "OPPORTUNISTIC",
|
|
|
+ "--enforce_execution_type"
|
|
|
+ };
|
|
|
+ client.init(args);
|
|
|
+ final AtomicBoolean result = new AtomicBoolean(false);
|
|
|
+ Thread t = new Thread() {
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ result.set(client.run());
|
|
|
+ } catch (Exception e) {
|
|
|
+ throw new RuntimeException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ t.start();
|
|
|
+
|
|
|
+ YarnClient yarnClient = YarnClient.createYarnClient();
|
|
|
+ yarnClient.init(new Configuration(yarnCluster.getConfig()));
|
|
|
+ yarnClient.start();
|
|
|
+ waitForContainersLaunch(yarnClient, 2);
|
|
|
+ List<ApplicationReport> apps = yarnClient.getApplications();
|
|
|
+ ApplicationReport appReport = apps.get(0);
|
|
|
+ ApplicationId appId = appReport.getApplicationId();
|
|
|
+ List<ApplicationAttemptReport> appAttempts =
|
|
|
+ yarnClient.getApplicationAttempts(appId);
|
|
|
+ ApplicationAttemptReport appAttemptReport = appAttempts.get(0);
|
|
|
+ ApplicationAttemptId appAttemptId =
|
|
|
+ appAttemptReport.getApplicationAttemptId();
|
|
|
+ List<ContainerReport> containers =
|
|
|
+ yarnClient.getContainers(appAttemptId);
|
|
|
+ // we should get two containers.
|
|
|
+ Assert.assertEquals(2, containers.size());
|
|
|
+ ContainerId amContainerId = appAttemptReport.getAMContainerId();
|
|
|
+ for (ContainerReport container : containers) {
|
|
|
+ if (!container.getContainerId().equals(amContainerId)) {
|
|
|
+ Assert.assertEquals(container.getExecutionType(),
|
|
|
+ ExecutionType.OPPORTUNISTIC);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } catch (Exception e) {
|
|
|
+ Assert.fail("Job execution with enforce execution type failed.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void waitForContainersLaunch(YarnClient client,
|
|
|
+ int nContainers) throws Exception {
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ public Boolean get() {
|
|
|
+ try {
|
|
|
+ List<ApplicationReport> apps = client.getApplications();
|
|
|
+ if (apps == null || apps.isEmpty()) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ ApplicationId appId = apps.get(0).getApplicationId();
|
|
|
+ List<ApplicationAttemptReport> appAttempts =
|
|
|
+ client.getApplicationAttempts(appId);
|
|
|
+ if (appAttempts == null || appAttempts.isEmpty()) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ ApplicationAttemptId attemptId =
|
|
|
+ appAttempts.get(0).getApplicationAttemptId();
|
|
|
+ List<ContainerReport> containers = client.getContainers(attemptId);
|
|
|
+ return (containers.size() == nContainers);
|
|
|
+ } catch (Exception e) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }, 10, 60000);
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
@TimelineVersion(2.0f)
|
|
|
public void testDistributedShellWithResources() throws Exception {
|