|
@@ -19,15 +19,20 @@
|
|
|
package org.apache.hadoop.yarn.submarine.client.cli.yarnservice;
|
|
|
|
|
|
import com.google.common.collect.ImmutableMap;
|
|
|
+import org.apache.hadoop.fs.FileUtil;
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.client.api.AppAdminClient;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Component;
|
|
|
+import org.apache.hadoop.yarn.service.api.records.ConfigFile;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Service;
|
|
|
import org.apache.hadoop.yarn.submarine.client.cli.RunJobCli;
|
|
|
import org.apache.hadoop.yarn.submarine.common.MockClientContext;
|
|
|
import org.apache.hadoop.yarn.submarine.common.api.TaskType;
|
|
|
+import org.apache.hadoop.yarn.submarine.common.conf.SubmarineConfiguration;
|
|
|
import org.apache.hadoop.yarn.submarine.common.conf.SubmarineLogs;
|
|
|
+import org.apache.hadoop.yarn.submarine.common.fs.RemoteDirectoryManager;
|
|
|
import org.apache.hadoop.yarn.submarine.runtimes.common.JobSubmitter;
|
|
|
import org.apache.hadoop.yarn.submarine.runtimes.common.StorageKeyConstants;
|
|
|
import org.apache.hadoop.yarn.submarine.runtimes.common.SubmarineStorage;
|
|
@@ -38,15 +43,22 @@ import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
+import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.nio.charset.Charset;
|
|
|
import java.nio.file.Files;
|
|
|
import java.nio.file.Paths;
|
|
|
+import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
|
import static org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes.EXIT_SUCCESS;
|
|
|
import static org.mockito.Matchers.any;
|
|
|
+import static org.mockito.Matchers.anyString;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.reset;
|
|
|
+import static org.mockito.Mockito.spy;
|
|
|
+import static org.mockito.Mockito.times;
|
|
|
+import static org.mockito.Mockito.verify;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
public class TestYarnServiceRunJobCli {
|
|
@@ -137,13 +149,13 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
"--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
"python run-job.py", "--worker_resources", "memory=2048M,vcores=2",
|
|
|
"--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
"ps.image", "--worker_docker_image", "worker.image",
|
|
|
- "--ps_launch_cmd", "python run-ps.py", "--verbose" });
|
|
|
+ "--ps_launch_cmd", "python run-ps.py", "--verbose"});
|
|
|
Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
runJobCli.getJobSubmitter());
|
|
|
Assert.assertEquals(3, serviceSpec.getComponents().size());
|
|
@@ -162,14 +174,14 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
"--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
"python run-job.py", "--worker_resources", "memory=2048M,vcores=2",
|
|
|
"--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
"ps.image", "--worker_docker_image", "worker.image",
|
|
|
"--tensorboard", "--ps_launch_cmd", "python run-ps.py",
|
|
|
- "--verbose" });
|
|
|
+ "--verbose"});
|
|
|
Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
runJobCli.getJobSubmitter());
|
|
|
Assert.assertEquals(4, serviceSpec.getComponents().size());
|
|
@@ -192,10 +204,10 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
"--num_workers", "1", "--worker_launch_cmd", "python run-job.py",
|
|
|
- "--worker_resources", "memory=2G,vcores=2", "--verbose" });
|
|
|
+ "--worker_resources", "memory=2G,vcores=2", "--verbose"});
|
|
|
|
|
|
Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
runJobCli.getJobSubmitter());
|
|
@@ -212,9 +224,9 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
- "--num_workers", "0", "--tensorboard", "--verbose" });
|
|
|
+ "--num_workers", "0", "--tensorboard", "--verbose"});
|
|
|
|
|
|
Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
runJobCli.getJobSubmitter());
|
|
@@ -233,11 +245,11 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
"--num_workers", "0", "--tensorboard", "--verbose",
|
|
|
"--tensorboard_resources", "memory=2G,vcores=2",
|
|
|
- "--tensorboard_docker_image", "tb_docker_image:001" });
|
|
|
+ "--tensorboard_docker_image", "tb_docker_image:001"});
|
|
|
|
|
|
Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
runJobCli.getJobSubmitter());
|
|
@@ -256,10 +268,10 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--num_workers", "0", "--tensorboard", "--verbose",
|
|
|
"--tensorboard_resources", "memory=2G,vcores=2",
|
|
|
- "--tensorboard_docker_image", "tb_docker_image:001" });
|
|
|
+ "--tensorboard_docker_image", "tb_docker_image:001"});
|
|
|
|
|
|
Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
runJobCli.getJobSubmitter());
|
|
@@ -307,7 +319,7 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertEquals(
|
|
|
runJobCli.getRunJobParameters().getTensorboardDockerImage(),
|
|
|
tensorboardComp.getArtifact().getId());
|
|
|
- } else{
|
|
|
+ } else {
|
|
|
Assert.assertNull(tensorboardComp.getArtifact());
|
|
|
}
|
|
|
|
|
@@ -352,11 +364,11 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
"--num_workers", "1", "--worker_launch_cmd", "python run-job.py",
|
|
|
"--worker_resources", "memory=2G,vcores=2", "--tensorboard",
|
|
|
- "--verbose" });
|
|
|
+ "--verbose"});
|
|
|
Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
runJobCli.getJobSubmitter());
|
|
|
|
|
@@ -376,10 +388,10 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--input_path", "s3://input", "--num_workers", "1",
|
|
|
"--worker_launch_cmd", "python run-job.py", "--worker_resources",
|
|
|
- "memory=2G,vcores=2", "--tensorboard", "--verbose" });
|
|
|
+ "memory=2G,vcores=2", "--tensorboard", "--verbose"});
|
|
|
Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
runJobCli.getJobSubmitter());
|
|
|
|
|
@@ -398,11 +410,11 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
"--num_workers", "1", "--worker_launch_cmd", "python run-job.py",
|
|
|
"--worker_resources", "memory=2G,vcores=2", "--tensorboard", "true",
|
|
|
- "--verbose" });
|
|
|
+ "--verbose"});
|
|
|
SubmarineStorage storage =
|
|
|
mockClientContext.getRuntimeFactory().getSubmarineStorage();
|
|
|
Map<String, String> jobInfo = storage.getJobInfoByName("my-job");
|
|
@@ -419,7 +431,7 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
"--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
"python run-job.py", "--worker_resources", "memory=2048M,vcores=2",
|
|
@@ -427,7 +439,7 @@ public class TestYarnServiceRunJobCli {
|
|
|
"ps.image", "--worker_docker_image", "worker.image",
|
|
|
"--ps_launch_cmd", "python run-ps.py", "--verbose", "--quicklink",
|
|
|
"AAA=http://master-0:8321", "--quicklink",
|
|
|
- "BBB=http://worker-0:1234" });
|
|
|
+ "BBB=http://worker-0:1234"});
|
|
|
Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
runJobCli.getJobSubmitter());
|
|
|
Assert.assertEquals(3, serviceSpec.getComponents().size());
|
|
@@ -447,7 +459,7 @@ public class TestYarnServiceRunJobCli {
|
|
|
Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
|
|
|
runJobCli.run(
|
|
|
- new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
"--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
"--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
"python run-job.py", "--worker_resources", "memory=2048M,vcores=2",
|
|
@@ -455,7 +467,7 @@ public class TestYarnServiceRunJobCli {
|
|
|
"ps.image", "--worker_docker_image", "worker.image",
|
|
|
"--ps_launch_cmd", "python run-ps.py", "--verbose", "--quicklink",
|
|
|
"AAA=http://master-0:8321", "--quicklink",
|
|
|
- "BBB=http://worker-0:1234", "--tensorboard" });
|
|
|
+ "BBB=http://worker-0:1234", "--tensorboard"});
|
|
|
Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
runJobCli.getJobSubmitter());
|
|
|
Assert.assertEquals(4, serviceSpec.getComponents().size());
|
|
@@ -468,4 +480,741 @@ public class TestYarnServiceRunJobCli {
|
|
|
YarnServiceJobSubmitter.TENSORBOARD_QUICKLINK_LABEL,
|
|
|
"http://tensorboard-0.my-job.username.null:6006"));
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Basic test.
|
|
|
+ * In one hand, create local temp file/dir for hdfs URI in
|
|
|
+ * local staging dir.
|
|
|
+ * In the other hand, use MockRemoteDirectoryManager mock
|
|
|
+ * implementation when check FileStatus or exists of HDFS file/dir
|
|
|
+ * --localization hdfs:///user/yarn/script1.py:.
|
|
|
+ * --localization /temp/script2.py:./
|
|
|
+ * --localization /temp/script2.py:/opt/script.py
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testRunJobWithBasicLocalization() throws Exception {
|
|
|
+ String remoteUrl = "hdfs:///user/yarn/script1.py";
|
|
|
+ String containerLocal1 = ".";
|
|
|
+ String localUrl = "/temp/script2.py";
|
|
|
+ String containerLocal2 = "./";
|
|
|
+ String containerLocal3 = "/opt/script.py";
|
|
|
+ String fakeLocalDir = System.getProperty("java.io.tmpdir");
|
|
|
+ // create local file, we need to put it under local temp dir
|
|
|
+ File localFile1 = new File(fakeLocalDir,
|
|
|
+ new Path(localUrl).getName());
|
|
|
+ localFile1.createNewFile();
|
|
|
+
|
|
|
+
|
|
|
+ MockClientContext mockClientContext =
|
|
|
+ YarnServiceCliTestUtils.getMockClientContext();
|
|
|
+ RunJobCli runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
+
|
|
|
+ RemoteDirectoryManager spyRdm =
|
|
|
+ spy(mockClientContext.getRemoteDirectoryManager());
|
|
|
+ mockClientContext.setRemoteDirectoryMgr(spyRdm);
|
|
|
+
|
|
|
+ // create remote file in local staging dir to simulate HDFS
|
|
|
+ Path stagingDir = mockClientContext.getRemoteDirectoryManager()
|
|
|
+ .getJobStagingArea("my-job", true);
|
|
|
+ File remoteFile1 = new File(stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUrl).getName());
|
|
|
+ remoteFile1.createNewFile();
|
|
|
+
|
|
|
+ Assert.assertTrue(localFile1.exists());
|
|
|
+ Assert.assertTrue(remoteFile1.exists());
|
|
|
+
|
|
|
+ runJobCli.run(
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ "--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
+ "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
+ "python run-job.py", "--worker_resources", "memory=2048M,vcores=2",
|
|
|
+ "--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
+ "ps.image", "--worker_docker_image", "worker.image",
|
|
|
+ "--ps_launch_cmd", "python run-ps.py", "--verbose",
|
|
|
+ "--localization",
|
|
|
+ remoteUrl + ":" + containerLocal1,
|
|
|
+ "--localization",
|
|
|
+ localFile1.getAbsolutePath() + ":" + containerLocal2,
|
|
|
+ "--localization",
|
|
|
+ localFile1.getAbsolutePath() + ":" + containerLocal3});
|
|
|
+ Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
+ runJobCli.getJobSubmitter());
|
|
|
+ Assert.assertEquals(3, serviceSpec.getComponents().size());
|
|
|
+
|
|
|
+ // No remote dir and hdfs file exists. Ensure download 0 times
|
|
|
+ verify(spyRdm, times(0)).copyRemoteToLocal(
|
|
|
+ anyString(), anyString());
|
|
|
+ // Ensure local original files are not deleted
|
|
|
+ Assert.assertTrue(localFile1.exists());
|
|
|
+
|
|
|
+ List<ConfigFile> files = serviceSpec.getConfiguration().getFiles();
|
|
|
+ Assert.assertEquals(3, files.size());
|
|
|
+ ConfigFile file = files.get(0);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.STATIC, file.getType());
|
|
|
+ String expectedSrcLocalization = remoteUrl;
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ file.getSrcFile());
|
|
|
+ String expectedDstFileName = new Path(remoteUrl).getName();
|
|
|
+ Assert.assertEquals(expectedDstFileName, file.getDestFile());
|
|
|
+
|
|
|
+ file = files.get(1);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.STATIC, file.getType());
|
|
|
+ expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(localUrl).getName();
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+ expectedDstFileName = new Path(localUrl).getName();
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+
|
|
|
+ file = files.get(2);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.STATIC, file.getType());
|
|
|
+ expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(localUrl).getName();
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+ expectedDstFileName = new Path(localUrl).getName();
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+
|
|
|
+ // Ensure env value is correct
|
|
|
+ String env = serviceSpec.getConfiguration().getEnv()
|
|
|
+ .get("YARN_CONTAINER_RUNTIME_DOCKER_MOUNTS");
|
|
|
+ String expectedMounts = new Path(containerLocal3).getName()
|
|
|
+ + ":" + containerLocal3 + ":rw";
|
|
|
+ Assert.assertTrue(env.contains(expectedMounts));
|
|
|
+
|
|
|
+ remoteFile1.delete();
|
|
|
+ localFile1.delete();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Non HDFS remote URI test.
|
|
|
+ * --localization https://a/b/1.patch:.
|
|
|
+ * --localization s3a://a/dir:/opt/mys3dir
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testRunJobWithNonHDFSRemoteLocalization() throws Exception {
|
|
|
+ String remoteUri1 = "https://a/b/1.patch";
|
|
|
+ String containerLocal1 = ".";
|
|
|
+ String remoteUri2 = "s3a://a/s3dir";
|
|
|
+ String containerLocal2 = "/opt/mys3dir";
|
|
|
+
|
|
|
+ MockClientContext mockClientContext =
|
|
|
+ YarnServiceCliTestUtils.getMockClientContext();
|
|
|
+ RunJobCli runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
+
|
|
|
+ RemoteDirectoryManager spyRdm =
|
|
|
+ spy(mockClientContext.getRemoteDirectoryManager());
|
|
|
+ mockClientContext.setRemoteDirectoryMgr(spyRdm);
|
|
|
+
|
|
|
+ // create remote file in local staging dir to simulate HDFS
|
|
|
+ Path stagingDir = mockClientContext.getRemoteDirectoryManager()
|
|
|
+ .getJobStagingArea("my-job", true);
|
|
|
+ File remoteFile1 = new File(stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUri1).getName());
|
|
|
+ remoteFile1.createNewFile();
|
|
|
+
|
|
|
+ File remoteDir1 = new File(stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUri2).getName());
|
|
|
+ remoteDir1.mkdir();
|
|
|
+ File remoteDir1File1 = new File(remoteDir1, "afile");
|
|
|
+ remoteDir1File1.createNewFile();
|
|
|
+
|
|
|
+ Assert.assertTrue(remoteFile1.exists());
|
|
|
+ Assert.assertTrue(remoteDir1.exists());
|
|
|
+ Assert.assertTrue(remoteDir1File1.exists());
|
|
|
+
|
|
|
+ String suffix1 = "_" + remoteDir1.lastModified()
|
|
|
+ + "-" + mockClientContext.getRemoteDirectoryManager()
|
|
|
+ .getRemoteFileSize(remoteUri2);
|
|
|
+ runJobCli.run(
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ "--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
+ "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
+ "python run-job.py", "--worker_resources", "memory=2048M,vcores=2",
|
|
|
+ "--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
+ "ps.image", "--worker_docker_image", "worker.image",
|
|
|
+ "--ps_launch_cmd", "python run-ps.py", "--verbose",
|
|
|
+ "--localization",
|
|
|
+ remoteUri1 + ":" + containerLocal1,
|
|
|
+ "--localization",
|
|
|
+ remoteUri2 + ":" + containerLocal2});
|
|
|
+ Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
+ runJobCli.getJobSubmitter());
|
|
|
+ Assert.assertEquals(3, serviceSpec.getComponents().size());
|
|
|
+
|
|
|
+ // Ensure download remote dir 2 times
|
|
|
+ verify(spyRdm, times(2)).copyRemoteToLocal(
|
|
|
+ anyString(), anyString());
|
|
|
+
|
|
|
+ // Ensure downloaded temp files are deleted
|
|
|
+ Assert.assertFalse(new File(System.getProperty("java.io.tmpdir")
|
|
|
+ + "/" + new Path(remoteUri1).getName()).exists());
|
|
|
+ Assert.assertFalse(new File(System.getProperty("java.io.tmpdir")
|
|
|
+ + "/" + new Path(remoteUri2).getName()).exists());
|
|
|
+
|
|
|
+ // Ensure zip file are deleted
|
|
|
+ Assert.assertFalse(new File(System.getProperty("java.io.tmpdir")
|
|
|
+ + "/" + new Path(remoteUri2).getName()
|
|
|
+ + "_" + suffix1 + ".zip").exists());
|
|
|
+
|
|
|
+ List<ConfigFile> files = serviceSpec.getConfiguration().getFiles();
|
|
|
+ Assert.assertEquals(2, files.size());
|
|
|
+ ConfigFile file = files.get(0);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.STATIC, file.getType());
|
|
|
+ String expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUri1).getName();
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+ String expectedDstFileName = new Path(remoteUri1).getName();
|
|
|
+ Assert.assertEquals(expectedDstFileName, file.getDestFile());
|
|
|
+
|
|
|
+ file = files.get(1);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.ARCHIVE, file.getType());
|
|
|
+ expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUri2).getName() + suffix1 + ".zip";
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+
|
|
|
+ expectedDstFileName = new Path(containerLocal2).getName();
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+
|
|
|
+ // Ensure env value is correct
|
|
|
+ String env = serviceSpec.getConfiguration().getEnv()
|
|
|
+ .get("YARN_CONTAINER_RUNTIME_DOCKER_MOUNTS");
|
|
|
+ String expectedMounts = new Path(remoteUri2).getName()
|
|
|
+ + ":" + containerLocal2 + ":rw";
|
|
|
+ Assert.assertTrue(env.contains(expectedMounts));
|
|
|
+
|
|
|
+ remoteDir1File1.delete();
|
|
|
+ remoteFile1.delete();
|
|
|
+ remoteDir1.delete();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test HDFS dir localization.
|
|
|
+ * --localization hdfs:///user/yarn/mydir:./mydir1
|
|
|
+ * --localization hdfs:///user/yarn/mydir2:/opt/dir2:rw
|
|
|
+ * --localization hdfs:///user/yarn/mydir:.
|
|
|
+ * --localization hdfs:///user/yarn/mydir2:./
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testRunJobWithHdfsDirLocalization() throws Exception {
|
|
|
+ String remoteUrl = "hdfs:///user/yarn/mydir";
|
|
|
+ String containerPath = "./mydir1";
|
|
|
+ String remoteUrl2 = "hdfs:///user/yarn/mydir2";
|
|
|
+ String containPath2 = "/opt/dir2";
|
|
|
+ String containerPath3 = ".";
|
|
|
+ String containerPath4 = "./";
|
|
|
+ MockClientContext mockClientContext =
|
|
|
+ YarnServiceCliTestUtils.getMockClientContext();
|
|
|
+ RunJobCli runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
+
|
|
|
+ RemoteDirectoryManager spyRdm =
|
|
|
+ spy(mockClientContext.getRemoteDirectoryManager());
|
|
|
+ mockClientContext.setRemoteDirectoryMgr(spyRdm);
|
|
|
+ // create remote file in local staging dir to simulate HDFS
|
|
|
+ Path stagingDir = mockClientContext.getRemoteDirectoryManager()
|
|
|
+ .getJobStagingArea("my-job", true);
|
|
|
+ File remoteDir1 = new File(stagingDir.toUri().getPath().toString()
|
|
|
+ + "/" + new Path(remoteUrl).getName());
|
|
|
+ remoteDir1.mkdir();
|
|
|
+ File remoteFile1 = new File(remoteDir1.getAbsolutePath() + "/1.py");
|
|
|
+ File remoteFile2 = new File(remoteDir1.getAbsolutePath() + "/2.py");
|
|
|
+ remoteFile1.createNewFile();
|
|
|
+ remoteFile2.createNewFile();
|
|
|
+
|
|
|
+ File remoteDir2 = new File(stagingDir.toUri().getPath().toString()
|
|
|
+ + "/" + new Path(remoteUrl2).getName());
|
|
|
+ remoteDir2.mkdir();
|
|
|
+ File remoteFile3 = new File(remoteDir1.getAbsolutePath() + "/3.py");
|
|
|
+ File remoteFile4 = new File(remoteDir1.getAbsolutePath() + "/4.py");
|
|
|
+ remoteFile3.createNewFile();
|
|
|
+ remoteFile4.createNewFile();
|
|
|
+
|
|
|
+ Assert.assertTrue(remoteDir1.exists());
|
|
|
+ Assert.assertTrue(remoteDir2.exists());
|
|
|
+
|
|
|
+ String suffix1 = "_" + remoteDir1.lastModified()
|
|
|
+ + "-" + mockClientContext.getRemoteDirectoryManager()
|
|
|
+ .getRemoteFileSize(remoteUrl);
|
|
|
+ String suffix2 = "_" + remoteDir2.lastModified()
|
|
|
+ + "-" + mockClientContext.getRemoteDirectoryManager()
|
|
|
+ .getRemoteFileSize(remoteUrl2);
|
|
|
+ runJobCli.run(
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ "--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
+ "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
+ "python run-job.py", "--worker_resources", "memory=2048M,vcores=2",
|
|
|
+ "--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
+ "ps.image", "--worker_docker_image", "worker.image",
|
|
|
+ "--ps_launch_cmd", "python run-ps.py", "--verbose",
|
|
|
+ "--localization",
|
|
|
+ remoteUrl + ":" + containerPath,
|
|
|
+ "--localization",
|
|
|
+ remoteUrl2 + ":" + containPath2 + ":rw",
|
|
|
+ "--localization",
|
|
|
+ remoteUrl + ":" + containerPath3,
|
|
|
+ "--localization",
|
|
|
+ remoteUrl2 + ":" + containerPath4});
|
|
|
+ Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
+ runJobCli.getJobSubmitter());
|
|
|
+ Assert.assertEquals(3, serviceSpec.getComponents().size());
|
|
|
+
|
|
|
+ // Ensure download remote dir 4 times
|
|
|
+ verify(spyRdm, times(4)).copyRemoteToLocal(
|
|
|
+ anyString(), anyString());
|
|
|
+
|
|
|
+ // Ensure downloaded temp files are deleted
|
|
|
+ Assert.assertFalse(new File(System.getProperty("java.io.tmpdir")
|
|
|
+ + "/" + new Path(remoteUrl).getName()).exists());
|
|
|
+ Assert.assertFalse(new File(System.getProperty("java.io.tmpdir")
|
|
|
+ + "/" + new Path(remoteUrl2).getName()).exists());
|
|
|
+ // Ensure zip file are deleted
|
|
|
+ Assert.assertFalse(new File(System.getProperty("java.io.tmpdir")
|
|
|
+ + "/" + new Path(remoteUrl).getName()
|
|
|
+ + suffix1 + ".zip").exists());
|
|
|
+ Assert.assertFalse(new File(System.getProperty("java.io.tmpdir")
|
|
|
+ + "/" + new Path(remoteUrl2).getName()
|
|
|
+ + suffix2 + ".zip").exists());
|
|
|
+
|
|
|
+ // Ensure files will be localized
|
|
|
+ List<ConfigFile> files = serviceSpec.getConfiguration().getFiles();
|
|
|
+ Assert.assertEquals(4, files.size());
|
|
|
+ ConfigFile file = files.get(0);
|
|
|
+ // The hdfs dir should be download and compress and let YARN to uncompress
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.ARCHIVE, file.getType());
|
|
|
+ String expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUrl).getName() + suffix1 + ".zip";
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+
|
|
|
+ // Relative path in container, but not "." or "./". Use its own name
|
|
|
+ String expectedDstFileName = new Path(containerPath).getName();
|
|
|
+ Assert.assertEquals(expectedDstFileName, file.getDestFile());
|
|
|
+
|
|
|
+ file = files.get(1);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.ARCHIVE, file.getType());
|
|
|
+ expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUrl2).getName() + suffix2 + ".zip";
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+
|
|
|
+ expectedDstFileName = new Path(containPath2).getName();
|
|
|
+ Assert.assertEquals(expectedDstFileName, file.getDestFile());
|
|
|
+
|
|
|
+ file = files.get(2);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.ARCHIVE, file.getType());
|
|
|
+ expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUrl).getName() + suffix1 + ".zip";
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+ // Relative path in container ".", use remote path name
|
|
|
+ expectedDstFileName = new Path(remoteUrl).getName();
|
|
|
+ Assert.assertEquals(expectedDstFileName, file.getDestFile());
|
|
|
+
|
|
|
+ file = files.get(3);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.ARCHIVE, file.getType());
|
|
|
+ expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUrl2).getName() + suffix2 + ".zip";
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+ // Relative path in container "./", use remote path name
|
|
|
+ expectedDstFileName = new Path(remoteUrl2).getName();
|
|
|
+ Assert.assertEquals(expectedDstFileName, file.getDestFile());
|
|
|
+
|
|
|
+ // Ensure mounts env value is correct. Add one mount string
|
|
|
+ String env = serviceSpec.getConfiguration().getEnv()
|
|
|
+ .get("YARN_CONTAINER_RUNTIME_DOCKER_MOUNTS");
|
|
|
+
|
|
|
+ String expectedMounts =
|
|
|
+ new Path(containPath2).getName() + ":" + containPath2 + ":rw";
|
|
|
+ Assert.assertTrue(env.contains(expectedMounts));
|
|
|
+
|
|
|
+ remoteFile1.delete();
|
|
|
+ remoteFile2.delete();
|
|
|
+ remoteFile3.delete();
|
|
|
+ remoteFile4.delete();
|
|
|
+ remoteDir1.delete();
|
|
|
+ remoteDir2.delete();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test if file/dir to be localized whose size exceeds limit.
|
|
|
+ * Max 10MB in configuration, mock remote will
|
|
|
+ * always return file size 100MB.
|
|
|
+ * This configuration will fail the job which has remoteUri
|
|
|
+ * But don't impact local dir/file
|
|
|
+ *
|
|
|
+ * --localization https://a/b/1.patch:.
|
|
|
+ * --localization s3a://a/dir:/opt/mys3dir
|
|
|
+ * --localization /temp/script2.py:./
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testRunJobRemoteUriExceedLocalizationSize() throws Exception {
|
|
|
+ String remoteUri1 = "https://a/b/1.patch";
|
|
|
+ String containerLocal1 = ".";
|
|
|
+ String remoteUri2 = "s3a://a/s3dir";
|
|
|
+ String containerLocal2 = "/opt/mys3dir";
|
|
|
+ String localUri1 = "/temp/script2";
|
|
|
+ String containerLocal3 = "./";
|
|
|
+
|
|
|
+ MockClientContext mockClientContext =
|
|
|
+ YarnServiceCliTestUtils.getMockClientContext();
|
|
|
+ SubmarineConfiguration submarineConf = new SubmarineConfiguration();
|
|
|
+ RemoteDirectoryManager spyRdm =
|
|
|
+ spy(mockClientContext.getRemoteDirectoryManager());
|
|
|
+ mockClientContext.setRemoteDirectoryMgr(spyRdm);
|
|
|
+ /**
|
|
|
+ * Max 10MB, mock remote will always return file size 100MB.
|
|
|
+ * */
|
|
|
+ submarineConf.set(
|
|
|
+ SubmarineConfiguration.LOCALIZATION_MAX_ALLOWED_FILE_SIZE_MB,
|
|
|
+ "10");
|
|
|
+ mockClientContext.setSubmarineConfig(submarineConf);
|
|
|
+
|
|
|
+ RunJobCli runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
+
|
|
|
+ // create remote file in local staging dir to simulate
|
|
|
+ Path stagingDir = mockClientContext.getRemoteDirectoryManager()
|
|
|
+ .getJobStagingArea("my-job", true);
|
|
|
+ File remoteFile1 = new File(stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUri1).getName());
|
|
|
+ remoteFile1.createNewFile();
|
|
|
+ File remoteDir1 = new File(stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(remoteUri2).getName());
|
|
|
+ remoteDir1.mkdir();
|
|
|
+
|
|
|
+ File remoteDir1File1 = new File(remoteDir1, "afile");
|
|
|
+ remoteDir1File1.createNewFile();
|
|
|
+
|
|
|
+ String fakeLocalDir = System.getProperty("java.io.tmpdir");
|
|
|
+ // create local file, we need to put it under local temp dir
|
|
|
+ File localFile1 = new File(fakeLocalDir,
|
|
|
+ new Path(localUri1).getName());
|
|
|
+ localFile1.createNewFile();
|
|
|
+
|
|
|
+ Assert.assertTrue(remoteFile1.exists());
|
|
|
+ Assert.assertTrue(remoteDir1.exists());
|
|
|
+ Assert.assertTrue(remoteDir1File1.exists());
|
|
|
+
|
|
|
+ String suffix1 = "_" + remoteDir1.lastModified()
|
|
|
+ + "-" + remoteDir1.length();
|
|
|
+ try {
|
|
|
+ runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ runJobCli.run(
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ "--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
+ "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
+ "python run-job.py", "--worker_resources",
|
|
|
+ "memory=2048M,vcores=2",
|
|
|
+ "--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
+ "ps.image", "--worker_docker_image", "worker.image",
|
|
|
+ "--ps_launch_cmd", "python run-ps.py", "--verbose",
|
|
|
+ "--localization",
|
|
|
+ remoteUri1 + ":" + containerLocal1});
|
|
|
+ } catch (IOException e) {
|
|
|
+ // Shouldn't have exception because it's within file size limit
|
|
|
+ Assert.assertFalse(true);
|
|
|
+ }
|
|
|
+ // we should download because fail fast
|
|
|
+ verify(spyRdm, times(1)).copyRemoteToLocal(
|
|
|
+ anyString(), anyString());
|
|
|
+ try {
|
|
|
+ // reset
|
|
|
+ reset(spyRdm);
|
|
|
+ runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ runJobCli.run(
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ "--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
+ "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
+ "python run-job.py", "--worker_resources",
|
|
|
+ "memory=2048M,vcores=2",
|
|
|
+ "--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
+ "ps.image", "--worker_docker_image", "worker.image",
|
|
|
+ "--ps_launch_cmd", "python run-ps.py", "--verbose",
|
|
|
+ "--localization",
|
|
|
+ remoteUri1 + ":" + containerLocal1,
|
|
|
+ "--localization",
|
|
|
+ remoteUri2 + ":" + containerLocal2,
|
|
|
+ "--localization",
|
|
|
+ localFile1.getAbsolutePath() + ":" + containerLocal3});
|
|
|
+ } catch (IOException e) {
|
|
|
+ Assert.assertTrue(e.getMessage()
|
|
|
+ .contains("104857600 exceeds configured max size:10485760"));
|
|
|
+ // we shouldn't do any download because fail fast
|
|
|
+ verify(spyRdm, times(0)).copyRemoteToLocal(
|
|
|
+ anyString(), anyString());
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ runJobCli.run(
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ "--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
+ "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
+ "python run-job.py", "--worker_resources",
|
|
|
+ "memory=2048M,vcores=2",
|
|
|
+ "--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
+ "ps.image", "--worker_docker_image", "worker.image",
|
|
|
+ "--ps_launch_cmd", "python run-ps.py", "--verbose",
|
|
|
+ "--localization",
|
|
|
+ localFile1.getAbsolutePath() + ":" + containerLocal3});
|
|
|
+ } catch (IOException e) {
|
|
|
+ Assert.assertTrue(e.getMessage()
|
|
|
+ .contains("104857600 exceeds configured max size:10485760"));
|
|
|
+ // we shouldn't do any download because fail fast
|
|
|
+ verify(spyRdm, times(0)).copyRemoteToLocal(
|
|
|
+ anyString(), anyString());
|
|
|
+ }
|
|
|
+
|
|
|
+ localFile1.delete();
|
|
|
+ remoteDir1File1.delete();
|
|
|
+ remoteFile1.delete();
|
|
|
+ remoteDir1.delete();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test remote Uri doesn't exist.
|
|
|
+ * */
|
|
|
+ @Test
|
|
|
+ public void testRunJobWithNonExistRemoteUri() throws Exception {
|
|
|
+ String remoteUri1 = "hdfs:///a/b/1.patch";
|
|
|
+ String containerLocal1 = ".";
|
|
|
+ String localUri1 = "/a/b/c";
|
|
|
+ String containerLocal2 = "./";
|
|
|
+ MockClientContext mockClientContext =
|
|
|
+ YarnServiceCliTestUtils.getMockClientContext();
|
|
|
+
|
|
|
+ RunJobCli runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
+
|
|
|
+ try {
|
|
|
+ runJobCli.run(
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ "--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
+ "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
+ "python run-job.py", "--worker_resources",
|
|
|
+ "memory=2048M,vcores=2",
|
|
|
+ "--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
+ "ps.image", "--worker_docker_image", "worker.image",
|
|
|
+ "--ps_launch_cmd", "python run-ps.py", "--verbose",
|
|
|
+ "--localization",
|
|
|
+ remoteUri1 + ":" + containerLocal1});
|
|
|
+ } catch (IOException e) {
|
|
|
+ Assert.assertTrue(e.getMessage()
|
|
|
+ .contains("doesn't exists"));
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ runJobCli.run(
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ "--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
+ "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
+ "python run-job.py", "--worker_resources",
|
|
|
+ "memory=2048M,vcores=2",
|
|
|
+ "--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
+ "ps.image", "--worker_docker_image", "worker.image",
|
|
|
+ "--ps_launch_cmd", "python run-ps.py", "--verbose",
|
|
|
+ "--localization",
|
|
|
+ localUri1 + ":" + containerLocal2});
|
|
|
+ } catch (IOException e) {
|
|
|
+ Assert.assertTrue(e.getMessage()
|
|
|
+ .contains("doesn't exists"));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test local dir
|
|
|
+ * --localization /user/yarn/mydir:./mydir1
|
|
|
+ * --localization /user/yarn/mydir2:/opt/dir2:rw
|
|
|
+ * --localization /user/yarn/mydir2:.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testRunJobWithLocalDirLocalization() throws Exception {
|
|
|
+ String fakeLocalDir = System.getProperty("java.io.tmpdir");
|
|
|
+ String localUrl = "/user/yarn/mydir";
|
|
|
+ String containerPath = "./mydir1";
|
|
|
+ String localUrl2 = "/user/yarn/mydir2";
|
|
|
+ String containPath2 = "/opt/dir2";
|
|
|
+ String containerPath3 = ".";
|
|
|
+
|
|
|
+ MockClientContext mockClientContext =
|
|
|
+ YarnServiceCliTestUtils.getMockClientContext();
|
|
|
+ RunJobCli runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ Assert.assertFalse(SubmarineLogs.isVerbose());
|
|
|
+
|
|
|
+ RemoteDirectoryManager spyRdm =
|
|
|
+ spy(mockClientContext.getRemoteDirectoryManager());
|
|
|
+ mockClientContext.setRemoteDirectoryMgr(spyRdm);
|
|
|
+ // create local file
|
|
|
+ File localDir1 = new File(fakeLocalDir,
|
|
|
+ localUrl);
|
|
|
+ localDir1.mkdirs();
|
|
|
+ File temp1 = new File(localDir1.getAbsolutePath() + "/1.py");
|
|
|
+ File temp2 = new File(localDir1.getAbsolutePath() + "/2.py");
|
|
|
+ temp1.createNewFile();
|
|
|
+ temp2.createNewFile();
|
|
|
+
|
|
|
+ File localDir2 = new File(fakeLocalDir,
|
|
|
+ localUrl2);
|
|
|
+ localDir2.mkdirs();
|
|
|
+ File temp3 = new File(localDir1.getAbsolutePath() + "/3.py");
|
|
|
+ File temp4 = new File(localDir1.getAbsolutePath() + "/4.py");
|
|
|
+ temp3.createNewFile();
|
|
|
+ temp4.createNewFile();
|
|
|
+
|
|
|
+ Assert.assertTrue(localDir1.exists());
|
|
|
+ Assert.assertTrue(localDir2.exists());
|
|
|
+
|
|
|
+ String suffix1 = "_" + localDir1.lastModified()
|
|
|
+ + "-" + localDir1.length();
|
|
|
+ String suffix2 = "_" + localDir2.lastModified()
|
|
|
+ + "-" + localDir2.length();
|
|
|
+
|
|
|
+ runJobCli.run(
|
|
|
+ new String[]{"--name", "my-job", "--docker_image", "tf-docker:1.1.0",
|
|
|
+ "--input_path", "s3://input", "--checkpoint_path", "s3://output",
|
|
|
+ "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
|
|
|
+ "python run-job.py", "--worker_resources", "memory=2048M,vcores=2",
|
|
|
+ "--ps_resources", "memory=4096M,vcores=4", "--ps_docker_image",
|
|
|
+ "ps.image", "--worker_docker_image", "worker.image",
|
|
|
+ "--ps_launch_cmd", "python run-ps.py", "--verbose",
|
|
|
+ "--localization",
|
|
|
+ fakeLocalDir + localUrl + ":" + containerPath,
|
|
|
+ "--localization",
|
|
|
+ fakeLocalDir + localUrl2 + ":" + containPath2 + ":rw",
|
|
|
+ "--localization",
|
|
|
+ fakeLocalDir + localUrl2 + ":" + containerPath3});
|
|
|
+
|
|
|
+ Service serviceSpec = getServiceSpecFromJobSubmitter(
|
|
|
+ runJobCli.getJobSubmitter());
|
|
|
+ Assert.assertEquals(3, serviceSpec.getComponents().size());
|
|
|
+
|
|
|
+ // we shouldn't do any download
|
|
|
+ verify(spyRdm, times(0)).copyRemoteToLocal(
|
|
|
+ anyString(), anyString());
|
|
|
+
|
|
|
+ // Ensure local original files are not deleted
|
|
|
+ Assert.assertTrue(localDir1.exists());
|
|
|
+ Assert.assertTrue(localDir2.exists());
|
|
|
+
|
|
|
+ // Ensure zip file are deleted
|
|
|
+ Assert.assertFalse(new File(System.getProperty("java.io.tmpdir")
|
|
|
+ + "/" + new Path(localUrl).getName()
|
|
|
+ + suffix1 + ".zip").exists());
|
|
|
+ Assert.assertFalse(new File(System.getProperty("java.io.tmpdir")
|
|
|
+ + "/" + new Path(localUrl2).getName()
|
|
|
+ + suffix2 + ".zip").exists());
|
|
|
+
|
|
|
+ // Ensure dirs will be zipped and localized
|
|
|
+ List<ConfigFile> files = serviceSpec.getConfiguration().getFiles();
|
|
|
+ Assert.assertEquals(3, files.size());
|
|
|
+ ConfigFile file = files.get(0);
|
|
|
+ Path stagingDir = mockClientContext.getRemoteDirectoryManager()
|
|
|
+ .getJobStagingArea("my-job", true);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.ARCHIVE, file.getType());
|
|
|
+ String expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(localUrl).getName() + suffix1 + ".zip";
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+ String expectedDstFileName = new Path(containerPath).getName();
|
|
|
+ Assert.assertEquals(expectedDstFileName, file.getDestFile());
|
|
|
+
|
|
|
+ file = files.get(1);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.ARCHIVE, file.getType());
|
|
|
+ expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(localUrl2).getName() + suffix2 + ".zip";
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+ expectedDstFileName = new Path(containPath2).getName();
|
|
|
+ Assert.assertEquals(expectedDstFileName, file.getDestFile());
|
|
|
+
|
|
|
+ file = files.get(2);
|
|
|
+ Assert.assertEquals(ConfigFile.TypeEnum.ARCHIVE, file.getType());
|
|
|
+ expectedSrcLocalization = stagingDir.toUri().getPath()
|
|
|
+ + "/" + new Path(localUrl2).getName() + suffix2 + ".zip";
|
|
|
+ Assert.assertEquals(expectedSrcLocalization,
|
|
|
+ new Path(file.getSrcFile()).toUri().getPath());
|
|
|
+ expectedDstFileName = new Path(localUrl2).getName();
|
|
|
+ Assert.assertEquals(expectedDstFileName, file.getDestFile());
|
|
|
+
|
|
|
+ // Ensure mounts env value is correct
|
|
|
+ String env = serviceSpec.getConfiguration().getEnv()
|
|
|
+ .get("YARN_CONTAINER_RUNTIME_DOCKER_MOUNTS");
|
|
|
+ String expectedMounts = new Path(containPath2).getName()
|
|
|
+ + ":" + containPath2 + ":rw";
|
|
|
+
|
|
|
+ Assert.assertTrue(env.contains(expectedMounts));
|
|
|
+
|
|
|
+ temp1.delete();
|
|
|
+ temp2.delete();
|
|
|
+ temp3.delete();
|
|
|
+ temp4.delete();
|
|
|
+ localDir2.delete();
|
|
|
+ localDir1.delete();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test zip function.
|
|
|
+ * A dir "/user/yarn/mydir" has two files and one subdir
|
|
|
+ * */
|
|
|
+ @Test
|
|
|
+ public void testYarnServiceSubmitterZipFunction()
|
|
|
+ throws Exception {
|
|
|
+ MockClientContext mockClientContext =
|
|
|
+ YarnServiceCliTestUtils.getMockClientContext();
|
|
|
+ RunJobCli runJobCli = new RunJobCli(mockClientContext);
|
|
|
+ YarnServiceJobSubmitter submitter =
|
|
|
+ (YarnServiceJobSubmitter)mockClientContext
|
|
|
+ .getRuntimeFactory().getJobSubmitterInstance();
|
|
|
+ String fakeLocalDir = System.getProperty("java.io.tmpdir");
|
|
|
+ String localUrl = "/user/yarn/mydir";
|
|
|
+ String localSubDirName = "subdir1";
|
|
|
+ // create local file
|
|
|
+ File localDir1 = new File(fakeLocalDir,
|
|
|
+ localUrl);
|
|
|
+ localDir1.mkdirs();
|
|
|
+ File temp1 = new File(localDir1.getAbsolutePath() + "/1.py");
|
|
|
+ File temp2 = new File(localDir1.getAbsolutePath() + "/2.py");
|
|
|
+ temp1.createNewFile();
|
|
|
+ temp2.createNewFile();
|
|
|
+
|
|
|
+
|
|
|
+ File localSubDir = new File(localDir1.getAbsolutePath(), localSubDirName);
|
|
|
+ localSubDir.mkdir();
|
|
|
+ File temp3 = new File(localSubDir.getAbsolutePath(), "3.py");
|
|
|
+ temp3.createNewFile();
|
|
|
+
|
|
|
+
|
|
|
+ String zipFilePath = submitter.zipDir(localDir1.getAbsolutePath(),
|
|
|
+ fakeLocalDir + "/user/yarn/mydir.zip");
|
|
|
+ File zipFile = new File(zipFilePath);
|
|
|
+ File unzipTargetDir = new File(fakeLocalDir, "unzipDir");
|
|
|
+ FileUtil.unZip(zipFile, unzipTargetDir);
|
|
|
+ Assert.assertTrue(
|
|
|
+ new File(fakeLocalDir + "/unzipDir/1.py").exists());
|
|
|
+ Assert.assertTrue(
|
|
|
+ new File(fakeLocalDir + "/unzipDir/2.py").exists());
|
|
|
+ Assert.assertTrue(
|
|
|
+ new File(fakeLocalDir + "/unzipDir/subdir1").exists());
|
|
|
+ Assert.assertTrue(
|
|
|
+ new File(fakeLocalDir + "/unzipDir/subdir1/3.py").exists());
|
|
|
+
|
|
|
+ zipFile.delete();
|
|
|
+ unzipTargetDir.delete();
|
|
|
+ temp1.delete();
|
|
|
+ temp2.delete();
|
|
|
+ temp3.delete();
|
|
|
+ localSubDir.delete();
|
|
|
+ localDir1.delete();
|
|
|
+ }
|
|
|
+
|
|
|
}
|