diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java index 8ca7a90..4f35ffb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java @@ -120,6 +120,11 @@ private CommandLine commandLine; private final boolean parseSuccessful; + // This is key used to pass libjars, files, archives to MRJobResourceUploader + public static final String TMP_LIBJARS_CONF_KEY = "tmpjars"; + public static final String TMP_ARCHIVES_CONF_KEY = "tmparchives"; + public static final String TMP_FILES_CONF_KEY = "tmpfiles"; + /** * Create an options parser with the given options to parse the args. * @param opts the options @@ -315,7 +320,7 @@ private void processGeneralOptions(CommandLine line) throws IOException { if (line.hasOption("libjars")) { // for libjars, we allow expansion of wildcards - conf.set("tmpjars", + conf.set(TMP_LIBJARS_CONF_KEY, validateFiles(line.getOptionValue("libjars"), true), "from -libjars command line option"); //setting libjars in client classpath @@ -328,12 +333,12 @@ private void processGeneralOptions(CommandLine line) throws IOException { } } if (line.hasOption("files")) { - conf.set("tmpfiles", + conf.set(TMP_FILES_CONF_KEY, validateFiles(line.getOptionValue("files")), "from -files command line option"); } if (line.hasOption("archives")) { - conf.set("tmparchives", + conf.set(TMP_ARCHIVES_CONF_KEY, validateFiles(line.getOptionValue("archives")), "from -archives command line option"); } @@ -402,11 +407,17 @@ private String validateFiles(String files) throws IOException { /** * takes input as a comma separated list of files + * with optional visibility setting + * {@see org.apache.hadoop.mapreduce.MRResourceVisibility#public, + * org.apache.hadoop.mapreduce.MRResourceVisibility#private, + * org.apache.hadoop.mapreduce.MRResourceVisibility#application} * and verifies if they exist. It defaults for file:/// * if the files specified do not have a scheme. * it returns the paths uri converted defaulting to file:///. * So an input of /home/user/file1,/home/user/file2 would return * file:///home/user/file1,file:///home/user/file2. + * An input of /home/user/file1::public,/home/user/file2::public would return + * file:///home/user/file1::public,file:///home/user/file2::public. * * @param files the input files argument * @param expandWildcard whether a wildcard entry is allowed and expanded. If @@ -427,52 +438,59 @@ private String validateFiles(String files, boolean expandWildcard) } List finalPaths = new ArrayList<>(fileArr.length); for (int i =0; i < fileArr.length; i++) { - String tmp = fileArr[i]; - if (tmp.isEmpty()) { + String rawResourceStr = fileArr[i].trim(); + if (rawResourceStr.isEmpty()) { throw new IllegalArgumentException("File name can't be empty string"); } + ResourceWithVisibilitySetting rawConfigResource + = ResourceWithVisibilitySetting.deserialize(rawResourceStr); + String resourceUri = rawConfigResource.getPathStr(); URI pathURI; final String wildcard = "*"; - boolean isWildcard = tmp.endsWith(wildcard) && expandWildcard; + boolean isWildcard = resourceUri.endsWith(wildcard) && expandWildcard; try { if (isWildcard) { // strip the wildcard - tmp = tmp.substring(0, tmp.length() - 1); + resourceUri = resourceUri.substring(0, resourceUri.length() - 1); } // handle the case where a wildcard alone ("*") or the wildcard on the // current directory ("./*") is specified - pathURI = matchesCurrentDirectory(tmp) ? + pathURI = matchesCurrentDirectory(resourceUri) ? new File(Path.CUR_DIR).toURI() : - new URI(tmp); + new URI(resourceUri); } catch (URISyntaxException e) { throw new IllegalArgumentException(e); } Path path = new Path(pathURI); - FileSystem localFs = FileSystem.getLocal(conf); - if (pathURI.getScheme() == null) { + FileSystem fs = FileSystem.getLocal(conf); + if (pathURI.getScheme() != null && pathURI.getScheme() != "file") { //default to the local file system //check if the file exists or not first - localFs.getFileStatus(path); - if (isWildcard) { - expandWildcard(finalPaths, path, localFs); - } else { - finalPaths.add(path.makeQualified(localFs.getUri(), - localFs.getWorkingDirectory()).toString()); + fs = path.getFileSystem(conf); + + } + // check if the file exists in this file system + // we need to recreate this filesystem object to copy + // these files to the file system ResourceManager is running + // on. + fs.getFileStatus(path); + if (isWildcard) { + List expandJarList = expandWildcard(path, fs); + String visibilitySettings = rawConfigResource.getVisibilitySettings(); + for (String pathStr : expandJarList) { + finalPaths.add(ResourceWithVisibilitySetting.serialize( + new ResourceWithVisibilitySetting(pathStr, visibilitySettings))); } } else { - // check if the file exists in this file system - // we need to recreate this filesystem object to copy - // these files to the file system ResourceManager is running - // on. - FileSystem fs = path.getFileSystem(conf); - // existence check - fs.getFileStatus(path); - if (isWildcard) { - expandWildcard(finalPaths, path, fs); - } else { - finalPaths.add(path.makeQualified(fs.getUri(), - fs.getWorkingDirectory()).toString()); - } + ResourceWithVisibilitySetting tmpResource = + new ResourceWithVisibilitySetting( + path.makeQualified( + fs.getUri(), fs.getWorkingDirectory()).toString(), + rawConfigResource.getVisibilitySettings()); + LOG.debug("Add tmp resource:" + + ResourceWithVisibilitySetting.serialize(tmpResource) + + ", uri:" + tmpResource.getPathStr()); + finalPaths.add(ResourceWithVisibilitySetting.serialize(tmpResource)); } } if (finalPaths.isEmpty()) { @@ -486,7 +504,7 @@ private boolean matchesCurrentDirectory(String path) { path.equals(Path.CUR_DIR + File.separator); } - private void expandWildcard(List finalPaths, Path path, FileSystem fs) + private List expandWildcard(Path path, FileSystem fs) throws IOException { FileStatus status = fs.getFileStatus(path); if (!status.isDirectory()) { @@ -497,12 +515,13 @@ private void expandWildcard(List finalPaths, Path path, FileSystem fs) fs.equals(FileSystem.getLocal(conf))); if (jars.isEmpty()) { LOG.warn(path + " does not have jars in it. It will be ignored."); - } else { - for (Path jar: jars) { - finalPaths.add(jar.makeQualified(fs.getUri(), - fs.getWorkingDirectory()).toString()); - } } + List res = new ArrayList<>(); + for (Path jarPath : jars) { + res.add(jarPath.makeQualified( + fs.getUri(), fs.getWorkingDirectory()).toString()); + } + return res; } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ResourceWithVisibilitySetting.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ResourceWithVisibilitySetting.java new file mode 100644 index 0000000..a57fe2a3 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ResourceWithVisibilitySetting.java @@ -0,0 +1,97 @@ +/** + * 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.util; + + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The class is used to represent one resource(libjar, archives or files) + * passed from cmd line. One resource require a path and optional + * visibility setting {See @org.apache.hadoop.MRResourceVisibility} + * E.g. -libjars example.jar::public or -libjars example.jar + * + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class ResourceWithVisibilitySetting { + public static final String PATH_SETTING_DELIMITER = "::"; + + private String pathStr; + private String visibilitySettings; + + private ResourceWithVisibilitySetting() { + } + + public String getPathStr() { + return pathStr; + } + + public String getVisibilitySettings() { + return visibilitySettings; + } + + public ResourceWithVisibilitySetting( + String pathString, String visibilitySettingsStr) { + pathStr = pathString; + visibilitySettings = visibilitySettingsStr; + } + + /** + * Serialize resource to string. + * @return $fileName:$settings + */ + public static String serialize( + ResourceWithVisibilitySetting resourceWithVisibilitySetting) { + if (resourceWithVisibilitySetting.visibilitySettings == null + || resourceWithVisibilitySetting.visibilitySettings.isEmpty()) { + return resourceWithVisibilitySetting.pathStr; + } + return resourceWithVisibilitySetting.pathStr + PATH_SETTING_DELIMITER + + resourceWithVisibilitySetting.visibilitySettings; + } + + /** + * Deserialize resource from string, the format should be + * {$fileName::$settings}. + * @param resourceStr + * @return ResourceWithVisibilitySetting + */ + public static ResourceWithVisibilitySetting deserialize(String resourceStr) { + String stripSchemaString = resourceStr; + // hdfs://exmaple.jar + if (!resourceStr.contains(PATH_SETTING_DELIMITER)) { + return new ResourceWithVisibilitySetting(resourceStr, null); + } + // hdfs://exmaple.jar::public + String[] strArr = stripSchemaString.split(PATH_SETTING_DELIMITER); + if (strArr.length != 2) { + throw new IllegalArgumentException( + "Resource format should be $fileName::$settings, " + resourceStr); + } + // public of hdfs://exmaple.jar::public + String settings = strArr[1]; + // hdfs://exmaple.jar + String uriPath = resourceStr.substring( + 0, resourceStr.length() + - PATH_SETTING_DELIMITER.length() - settings.length()); + return new ResourceWithVisibilitySetting(uriPath, settings); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java index 0dbfe3d..a894d5c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java @@ -164,6 +164,16 @@ public void testEmptyFilenames() throws Exception { GenericTestUtils.assertExceptionContains("File name can't be" + " empty string", e); } + args[1] = String.format("%s, ,%s", + tmpFileOne.toURI().toString(), tmpFileTwo.toURI().toString()); + try { + new GenericOptionsParser(conf, args); + fail("Expected exception for empty filename"); + } catch (IllegalArgumentException e) { + // expect to receive an IllegalArgumentException + GenericTestUtils.assertExceptionContains("File name can't be" + + " empty string", e); + } // test zero file list length - it should create an exception args[1] = ",,"; @@ -176,10 +186,10 @@ public void testEmptyFilenames() throws Exception { + " empty string", e); } - // test filename with space character + // test filename with special character // it should create exception from parser in URI class // due to URI syntax error - args[1] = String.format("%s, ,%s", + args[1] = String.format("%s,^,%s", tmpFileOne.toURI().toString(), tmpFileTwo.toURI().toString()); try { new GenericOptionsParser(conf, args); @@ -392,4 +402,135 @@ public void testNullArgs() throws IOException { GenericOptionsParser parser = new GenericOptionsParser(conf, null); parser.getRemainingArgs(); } + + @Test + public void testFilesWithVisibilitySettings() throws Exception { + testLocalResourceWithVisibilitySettings( + "-files", GenericOptionsParser.TMP_FILES_CONF_KEY); + } + + @Test + public void testLibjarsWithVisibilitySettings() throws Exception { + testLocalResourceWithVisibilitySettings( + "-libjars", GenericOptionsParser.TMP_LIBJARS_CONF_KEY); + } + + @Test + public void testLibjarsWildCardWithVisibilitySettings() throws Exception { + Path tmpPath0 = new Path( + new File(testDir, "testLibjarsWildCard0.JAR").toString()); + Path tmpPath1 = new Path( + new File(testDir, "testLibjarsWildCard1.JAR").toString()); + localFs.create(tmpPath0); + localFs.create(tmpPath1); + Configuration config = new Configuration(); + String[] args = new String[2]; + args[0] = "-libjars"; + args[1] = tmpPath0.getParent().toUri() + "/*" + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + "public"; + new GenericOptionsParser(config, args); + String[] resources = config.getStrings( + GenericOptionsParser.TMP_LIBJARS_CONF_KEY); + assertEquals("Reousrce number is 2", 2, resources.length); + assertEquals("First resource is not as expected", + localFs.makeQualified(tmpPath0).toString() + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + "public", resources[0]); + assertEquals("First resource is not as expected", + localFs.makeQualified(tmpPath1).toString() + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + "public", resources[1]); + } + + @Test + public void testArchivesWithVisibilitySettings() throws Exception { + testLocalResourceWithVisibilitySettings( + "-archives", GenericOptionsParser.TMP_ARCHIVES_CONF_KEY); + } + + public void testLocalResourceWithVisibilitySettings( + String userFacingConfigKey, String resourceConfigKey) + throws Exception { + File tmpFile = new File( + testDir, "testLocalResourceWithVisibilitySettingsTmpFile" + + userFacingConfigKey); + Path tmpPath = new Path(tmpFile.toString()); + localFs.create(tmpPath); + String[] args = new String[2]; + args[0] = userFacingConfigKey; + args[1] = tmpFile.toURI().toString() + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + "public"; + new GenericOptionsParser(conf, args); + String resources = conf.get(resourceConfigKey); + assertEquals("resources option does not match", + localFs.makeQualified(tmpPath).toString() + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + "public", resources); + + // pass file as uri + Configuration conf1 = new Configuration(); + URI tmpURI = new URI(tmpFile.toURI().toString() + "#link"); + args[0] = userFacingConfigKey; + args[1] = tmpURI.toString() + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + "application"; + new GenericOptionsParser(conf1, args); + resources = conf1.get(resourceConfigKey); + assertEquals("resources option does not match", + localFs.makeQualified( + new Path(tmpURI)).toString() + "::application", resources); + + Configuration conf2 = new Configuration(); + args[0] = userFacingConfigKey; + args[1] = "file:///xyz.txt::user"; + testBadConfigWithExceptions( + conf2, args, resourceConfigKey, FileNotFoundException.class); + + // Test bad configs + Configuration conf3 = new Configuration(); + args[0] = userFacingConfigKey; + args[1] = tmpFile.toURI().toString() + + ":public"; + testBadConfigWithExceptions( + conf3, args, resourceConfigKey, FileNotFoundException.class); + + // Test multiple files + // pass file as uri + File tmpFile1 = new File( + testDir, "testLocalResourceWithVisibilitySettingsTmpFile1"); + Path tmpPath1 = new Path(tmpFile1.toString()); + localFs.create(tmpPath1); + Configuration conf4 = new Configuration(); + args[0] = userFacingConfigKey; + args[1] = tmpFile.toString() + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + "application" + + "," + tmpFile1.toString() + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + "public"; + new GenericOptionsParser(conf4, args); + resources = conf4.get(resourceConfigKey); + assertEquals("resources option does not match", + localFs.makeQualified( + tmpPath).toString() + "::application," + + localFs.makeQualified(tmpPath1).toString() + "::public", + resources); + } + + private void testBadConfigWithExceptions( + Configuration config, String[] args, + String resourceConfigKey, Class expectEx) { + Throwable th = null; + try { + new GenericOptionsParser(config, args); + } catch (Exception e) { + th = e; + } + assertNotNull("throwable is null", th); + assertTrue( + expectEx.getClass().getName() + " exception is not thrown", + th.getClass().equals(expectEx)); + assertNull("resources is not null", config.get(resourceConfigKey)); + + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestResourceWithVisibilitySetting.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestResourceWithVisibilitySetting.java new file mode 100644 index 0000000..7cc59bf --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestResourceWithVisibilitySetting.java @@ -0,0 +1,90 @@ +/** + * 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.util; + +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; + +/** + * Test MR resource with visibility settings. + */ +public class TestResourceWithVisibilitySetting { + @Test + public void testSerialize() { + String path0 = "path0"; + String visSettings0 = "public"; + ResourceWithVisibilitySetting item0 = + new ResourceWithVisibilitySetting(path0, visSettings0); + assertTrue("Serialized string should be" + path0 + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + visSettings0 + ", but turn out be " + + ResourceWithVisibilitySetting.serialize(item0), + ResourceWithVisibilitySetting.serialize(item0). + equals(path0 + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + visSettings0)); + + String path1 = "path1"; + String visSettings1 = ""; + ResourceWithVisibilitySetting item1 = + new ResourceWithVisibilitySetting(path1, visSettings1); + assertTrue("Serialized string should be " + path1 + ", but is:" + + ResourceWithVisibilitySetting.serialize(item1), + ResourceWithVisibilitySetting.serialize(item1).equals(path1)); + + String path2 = "path2"; + ResourceWithVisibilitySetting item2 = + new ResourceWithVisibilitySetting(path2, null); + assertTrue("Serialized string should be " + path2 + ", but is:" + + ResourceWithVisibilitySetting.serialize(item2), + ResourceWithVisibilitySetting.serialize(item2).equals(path2)); + } + + @Test + public void testDeserialize() { + String path0 = "hdfs://nn/mr-tmp"; + ResourceWithVisibilitySetting item0 = + ResourceWithVisibilitySetting.deserialize(path0); + assertTrue("Path should " + path0 + + ", but is " + item0.getPathStr(), + item0.getPathStr().equals(path0)); + assertNull("Settings is not null", + item0.getVisibilitySettings()); + + String path1 = "hdfs://nn/mr-tmp::public"; + ResourceWithVisibilitySetting item1 = + ResourceWithVisibilitySetting.deserialize(path1); + assertTrue("Path should be hdfs://nn/mr-tmp, but is " + + item1.getPathStr(), + item1.getPathStr().equals("hdfs://nn/mr-tmp")); + assertTrue("Setting should be public", + item1.getVisibilitySettings().equals("public")); + + String path2 = "file:///file1::application"; + ResourceWithVisibilitySetting item2 = + ResourceWithVisibilitySetting.deserialize(path2); + assertTrue("Path should be file:///file1, but is " + + item2.getPathStr(), + item2.getPathStr().equals("file:///file1")); + assertTrue("Setting should be application", + item2.getVisibilitySettings().equals("application")); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java index d912b60..b3a3d80 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java @@ -72,6 +72,7 @@ import org.apache.hadoop.mapreduce.jobhistory.TaskAttemptUnsuccessfulCompletionEvent; import org.apache.hadoop.mapreduce.security.TokenCache; import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier; +import org.apache.hadoop.mapreduce.util.MRResourceUtil; import org.apache.hadoop.mapreduce.v2.api.records.Avataar; import org.apache.hadoop.mapreduce.v2.api.records.Locality; import org.apache.hadoop.mapreduce.v2.api.records.Phase; @@ -970,10 +971,7 @@ private static void configureJobJar(Configuration conf, Path remoteJobJar = jobJarPath.makeQualified(jobJarFs.getUri(), jobJarFs.getWorkingDirectory()); LocalResourceVisibility jobJarViz = - conf.getBoolean(MRJobConfig.JOBJAR_VISIBILITY, - MRJobConfig.JOBJAR_VISIBILITY_DEFAULT) - ? LocalResourceVisibility.PUBLIC - : LocalResourceVisibility.APPLICATION; + MRResourceUtil.getJobJarYarnLocalVisibility(conf); // We hard code the job.jar localized symlink in the container directory. // This is because the mapreduce app expects the job.jar to be named // accordingly. Additionally we set the shared cache upload policy to diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/LocalResourceBuilder.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/LocalResourceBuilder.java index 48cc29e..fa3074a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/LocalResourceBuilder.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/LocalResourceBuilder.java @@ -52,7 +52,7 @@ private URI[] uris; private long[] timestamps; private long[] sizes; - private boolean[] visibilities; + private LocalResourceVisibility[] visibilities; private Map sharedCacheUploadPolicies; LocalResourceBuilder() { @@ -78,7 +78,7 @@ void setSizes(long[] s) { this.sizes = s; } - void setVisibilities(boolean[] v) { + void setVisibilities(LocalResourceVisibility[] v) { this.visibilities = v; } @@ -108,8 +108,13 @@ void createLocalResources(Map localResources) String linkName = null; if (p.getName().equals(DistributedCache.WILDCARD)) { + // We won't have two jars/files share same name. + // However, the dirs might share same. e.g. + // job_***/public/libjars and job_***/user/libjars p = p.getParent(); - linkName = p.getName() + Path.SEPARATOR + DistributedCache.WILDCARD; + linkName = p.getParent().getName() + + Path.SEPARATOR + p.getName() + + Path.SEPARATOR + DistributedCache.WILDCARD; } p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(), @@ -151,8 +156,7 @@ void createLocalResources(Map localResources) sharedCachePolicy = sharedCachePolicy == null ? Boolean.FALSE : sharedCachePolicy; localResources.put(linkName, LocalResource.newInstance(URL.fromURI(p - .toUri()), type, visibilities[i] ? LocalResourceVisibility.PUBLIC - : LocalResourceVisibility.PRIVATE, + .toUri()), type, visibilities[i], sizes[i], timestamps[i], sharedCachePolicy)); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java index 1b087a7..a0dd272 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java @@ -137,7 +137,8 @@ public boolean correspondsTo(TaskAttemptState state) { RUNNING( new TaskState[]{TaskState.RUNNING}), PENDING(new TaskState[]{TaskState.SCHEDULED}), - COMPLETED(new TaskState[]{TaskState.SUCCEEDED, TaskState.FAILED, TaskState.KILLED}); + COMPLETED(new TaskState[]{ + TaskState.SUCCEEDED, TaskState.FAILED, TaskState.KILLED}); private final List correspondingStates; @@ -439,11 +440,31 @@ public ClassLoader run() throws MalformedURLException { } } - private static final String STAGING_CONSTANT = ".staging"; public static Path getStagingAreaDir(Configuration conf, String user) { - return new Path(conf.get(MRJobConfig.MR_AM_STAGING_DIR, - MRJobConfig.DEFAULT_MR_AM_STAGING_DIR) - + Path.SEPARATOR + user + Path.SEPARATOR + STAGING_CONSTANT); + // We will check MAPREDUCE_JOB_DIR first to prevent + // inconsistent staging dir between MR job client + // and application master. E.g. user used an old client + // which refers to /user/test/.staging + // and the application master which links a new version jar might + // refer /user/test/.newstaging + // as staging dir. We should refer to "/user/test/.staging" set by client. + // So we will check MAPREDUCE_JOB_DIR to see whether + // the configuration had been set by client. + // If it's set we should refer the existing staging dir + // instead of combining the parameters. + String jobDirPath = conf.get(MRJobConfig.MAPREDUCE_JOB_DIR); + if (jobDirPath != null) { + // The job dir path is already set by MR client. The format should be like + // ${stagingDir}/job_1234XXX + return new Path(jobDirPath).getParent(); + } + return new Path( + conf.get( + MRJobConfig.MR_AM_STAGING_DIR, + MRJobConfig.DEFAULT_MR_AM_STAGING_DIR) + + Path.SEPARATOR + user + Path.SEPARATOR + + conf.get(MRJobConfig.MR_USER_STAGING_DIR_NAME, + MRJobConfig.DEFAULT_MR_USER_STAGING_DIR_NAME)); } public static String getJobFile(Configuration conf, String user, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java index c6a2874..5bef74a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java @@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.junit.AfterClass; @@ -186,6 +187,43 @@ public void testGetJobFileWithUser() { "/my/path/to/staging/dummy-user/.staging/job_dummy-job_12345/job.xml", jobFile); } + @Test(timeout = 60000) + public void testGetStagingAreaDirDefault() { + Configuration conf = new Configuration(); + conf.set(MRJobConfig.MR_AM_STAGING_DIR, "/testGetStagingAreaDirDefault"); + String user = "dummy-user"; + Path stagingPath = MRApps.getStagingAreaDir(conf, user); + assertEquals("Staging path is not as expected", + "/testGetStagingAreaDirDefault/dummy-user/.staging", + stagingPath.toString()); + } + + + @Test(timeout = 60000) + public void testGetStagingAreaDirWithJobDirSet() { + Configuration conf = new Configuration(); + conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, + "/testGetStagingAreaDirWithJobDirSet/dummyUser/.staging2/job_123"); + conf.set(MRJobConfig.MR_AM_STAGING_DIR, "/stagingDir"); + String user = "dummy-user"; + Path stagingPath = MRApps.getStagingAreaDir(conf, user); + assertEquals("Staging path is not as expected", + "/testGetStagingAreaDirWithJobDirSet/dummyUser/.staging2", + stagingPath.toString()); + } + + @Test(timeout = 60000) + public void testGetStagingAreaDirNonDefault() { + Configuration conf = new Configuration(); + conf.set(MRJobConfig.MR_AM_STAGING_DIR, "/testGetStagingAreaDirDefault"); + conf.set(MRJobConfig.MR_USER_STAGING_DIR_NAME, ".newstaging"); + String user = "dummy-user"; + Path stagingPath = MRApps.getStagingAreaDir(conf, user); + assertEquals("Staging path is not as expected", + "/testGetStagingAreaDirDefault/dummy-user/.newstaging", + stagingPath.toString()); + } + @Test (timeout = 120000) public void testSetClasspath() throws IOException { Configuration conf = new Configuration(); @@ -380,11 +418,13 @@ public void testSetupDistributedCacheConflicts() throws Exception { DistributedCache.addCacheArchive(archive, conf); conf.set(MRJobConfig.CACHE_ARCHIVES_TIMESTAMPS, "10"); conf.set(MRJobConfig.CACHE_ARCHIVES_SIZES, "10"); - conf.set(MRJobConfig.CACHE_ARCHIVES_VISIBILITIES, "true"); + conf.set(MRJobConfig.CACHE_ARCHIVES_VISIBILITIES, + LocalResourceVisibility.APPLICATION.name()); DistributedCache.addCacheFile(file, conf); conf.set(MRJobConfig.CACHE_FILE_TIMESTAMPS, "11"); conf.set(MRJobConfig.CACHE_FILES_SIZES, "11"); - conf.set(MRJobConfig.CACHE_FILE_VISIBILITIES, "true"); + conf.set(MRJobConfig.CACHE_FILE_VISIBILITIES, + LocalResourceVisibility.PUBLIC.name()); Map localResources = new HashMap(); MRApps.setupDistributedCache(conf, localResources); @@ -396,6 +436,8 @@ public void testSetupDistributedCacheConflicts() throws Exception { assertEquals(10l, lr.getSize()); assertEquals(10l, lr.getTimestamp()); assertEquals(LocalResourceType.ARCHIVE, lr.getType()); + assertTrue(lr.getVisibility().name().equals( + LocalResourceVisibility.PUBLIC.name())); } @SuppressWarnings("deprecation") @@ -420,7 +462,9 @@ public void testSetupDistributedCacheConflictsFiles() throws Exception { DistributedCache.addCacheFile(file2, conf); conf.set(MRJobConfig.CACHE_FILE_TIMESTAMPS, "10,11"); conf.set(MRJobConfig.CACHE_FILES_SIZES, "10,11"); - conf.set(MRJobConfig.CACHE_FILE_VISIBILITIES, "true,true"); + conf.set(MRJobConfig.CACHE_FILE_VISIBILITIES, + LocalResourceVisibility.PUBLIC.name() + "," + + LocalResourceVisibility.PUBLIC.name()); Map localResources = new HashMap(); MRApps.setupDistributedCache(conf, localResources); @@ -455,11 +499,13 @@ public void testSetupDistributedCache() throws Exception { DistributedCache.addCacheArchive(archive, conf); conf.set(MRJobConfig.CACHE_ARCHIVES_TIMESTAMPS, "10"); conf.set(MRJobConfig.CACHE_ARCHIVES_SIZES, "10"); - conf.set(MRJobConfig.CACHE_ARCHIVES_VISIBILITIES, "true"); + conf.set(MRJobConfig.CACHE_ARCHIVES_VISIBILITIES, + LocalResourceVisibility.APPLICATION.name()); DistributedCache.addCacheFile(file, conf); conf.set(MRJobConfig.CACHE_FILE_TIMESTAMPS, "11"); conf.set(MRJobConfig.CACHE_FILES_SIZES, "11"); - conf.set(MRJobConfig.CACHE_FILE_VISIBILITIES, "true"); + conf.set(MRJobConfig.CACHE_FILE_VISIBILITIES, + LocalResourceVisibility.APPLICATION.name()); Map localResources = new HashMap(); MRApps.setupDistributedCache(conf, localResources); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java index 31e2057..a5c35ba 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java @@ -22,9 +22,11 @@ import java.net.URI; import java.security.PrivilegedExceptionAction; import java.util.Collection; +import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.Map; +import java.util.Set; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -1134,16 +1136,20 @@ public void setCacheFiles(URI[] files) { } /** - * Add a archives to be localized + * Add a archives to be localized. Please use + * {@link #addArchiveToSharedCache(URI, Configuration)} + * instead if possible. * @param uri The uri of the cache to be localized */ public void addCacheArchive(URI uri) { ensureState(JobState.DEFINE); DistributedCache.addCacheArchive(uri, conf); } - + /** - * Add a file to be localized + * Add a file to be localized. Please use + * @see #addFileToSharedCache(URI, Configuration) + * instead if it satisfies your need. * @param uri The uri of the cache to be localized */ public void addCacheFile(URI uri) { @@ -1180,7 +1186,8 @@ public void addFileToClassPath(Path file) public void addArchiveToClassPath(Path archive) throws IOException { ensureState(JobState.DEFINE); - DistributedCache.addArchiveToClassPath(archive, conf, archive.getFileSystem(conf)); + DistributedCache.addArchiveToClassPath( + archive, conf, archive.getFileSystem(conf), true); } /** @@ -1416,6 +1423,21 @@ public static void setFileSharedCacheUploadPolicies(Configuration conf, } /** + * This is to append the shared cache upload policies + * for files (including libjars) to existing settings. + * + * @param conf Configuration which stores the shared cache upload policies + * @param policies A map containing the shared cache upload policies for a set + * of resources. The key is the url of the resource and the value is + * the upload policy. True if it should be uploaded, false otherwise. + */ + @Unstable + public static void appendFileSharedCacheUploadPolicies( + Configuration conf, Map policies) { + appendSharedCacheUploadPolicies(conf, policies, true); + } + + /** * This is to set the shared cache upload policies for archives. If the * parameter was previously set, this method will replace the old value with * the new provided map. @@ -1472,6 +1494,27 @@ private static void setSharedCacheUploadPolicies(Configuration conf, } } + private static void appendSharedCacheUploadPolicies( + Configuration conf, Map policies, boolean areFiles) { + Map existingPolicies = + getSharedCacheUploadPolicies(conf, areFiles); + Set s = new HashSet(existingPolicies.keySet()); + s.retainAll(policies.keySet()); + for (String key : s) { + String msg = key + + "'s retention policy is already in config! Existing policy:" + + existingPolicies.get(key) + ", new policy:" + policies.get(key); + if (existingPolicies.get(key) != policies.get(key)) { + LOG.error(msg); + throw new IllegalArgumentException(msg); + } else { + msg += ", will ignore the appened op."; + LOG.warn(msg); + } + } + policies.putAll(existingPolicies); + setSharedCacheUploadPolicies(conf, policies, areFiles); + } /** * Deserialize a map of shared cache upload policies from a config parameter. * diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java index e106a54..385c634 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java @@ -21,10 +21,11 @@ import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; +import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.LinkedList; -import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -41,7 +42,9 @@ import org.apache.hadoop.ipc.RpcNoSuchMethodException; import org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager; import org.apache.hadoop.mapreduce.filecache.DistributedCache; +import org.apache.hadoop.mapreduce.util.MRResourceUtil; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.client.api.SharedCacheClient; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -64,12 +67,96 @@ private SharedCacheClient scClient = null; private SharedCacheConfig scConfig = new SharedCacheConfig(); private ApplicationId appId = null; + private Map statCache = new HashMap<>(); JobResourceUploader(FileSystem submitFs, boolean useWildcard) { this.jtFs = submitFs; this.useWildcard = useWildcard; } + public class MRResourceInfo { + private MRResource resource; + private URI uriToUseInJob; + private FileStatus fileToUseStatus; + private boolean isFromShareCache; + + private MRResourceInfo() {} + + MRResourceInfo( + MRResource mrResource, URI resourceToUse, boolean fromShareCache) { + resource = mrResource; + isFromShareCache = fromShareCache; + uriToUseInJob = resourceToUse; + fileToUseStatus = null; + } + + @VisibleForTesting + void initFileStatus(Configuration conf, Map fileStatusMap) + throws IOException { + // No lock here as we assume one instance won't be accessed concurrently. + if (fileToUseStatus == null) { + fileToUseStatus = ClientDistributedCacheManager.getFileStatus( + conf, uriToUseInJob, fileStatusMap); + } + } + + public long getUriToUseTimestamp( + Configuration conf, Map fileStatusMap) + throws IOException { + initFileStatus(conf, fileStatusMap); + return fileToUseStatus.getModificationTime(); + } + + public long getUriToUseFileSize( + Configuration conf, Map fileStatusMap) + throws IOException { + initFileStatus(conf, fileStatusMap); + return fileToUseStatus.getLen(); + } + + public boolean getUploadPolicy(SharedCacheConfig sharedCacheConfig) { + return determineResourceUploadPolicy( + sharedCacheConfig, resource, isFromShareCache); + } + + public LocalResourceVisibility getYarnResourceVisibility() { + return resource.getYarnLocalResourceVisibility(); + } + + void removeFragmentInUri() { + String fragment = uriToUseInJob.getFragment(); + if (fragment == null) { + return; + } + String uriStr = uriToUseInJob.toString(); + // hdfs://nn/jar#class1 => hdfs://nn/jar + uriStr = uriStr.substring(0, uriStr.length() - fragment.length() - 1); + try { + uriToUseInJob = new URI(uriStr); + } catch (URISyntaxException ex) { + throw new IllegalArgumentException("Should be impossible," + + "Wrong uri " + uriToUseInJob); + } + } + + @Override + public String toString() { + return resource.toString() + ", uriToUseInJob:" + uriToUseInJob + + ", uriToUseInJobStatus" + fileToUseStatus.toString(); + } + } + + public static boolean determineResourceUploadPolicy( + SharedCacheConfig scConf, MRResource resource, boolean fromSharedCache) { + if (!isSharedCacheEnabled(scConf, resource) + || resource.getResourceVisibility() != MRResourceVisibility.PUBLIC + || fromSharedCache) { + // Only public resource is support for the moment. + return false; + } + return true; + } + private void initSharedCache(JobID jobid, Configuration conf) { this.scConfig.init(conf); if (this.scConfig.isSharedCacheEnabled()) { @@ -136,19 +223,8 @@ public void uploadResources(Job job, Path submitJobDir) throws IOException { } } - private void uploadResourcesInternal(Job job, Path submitJobDir) + private void initJobAndResourceDirs(Job job, Path submitJobDir) throws IOException { - Configuration conf = job.getConfiguration(); - short replication = - (short) conf.getInt(Job.SUBMIT_REPLICATION, - Job.DEFAULT_SUBMIT_REPLICATION); - - if (!(conf.getBoolean(Job.USED_GENERIC_PARSER, false))) { - LOG.warn("Hadoop command-line option parsing not performed. " - + "Implement the Tool interface and execute your application " - + "with ToolRunner to remedy this."); - } - // // Figure out what fs the JobTracker is using. Copy the // job to it, under a temporary name. This allows DFS to work, @@ -164,322 +240,435 @@ private void uploadResourcesInternal(Job job, Path submitJobDir) + " already exists!! This is unexpected.Please check what's there in" + " that directory"); } + /** + * Current job staging dir structure is + * ${yarn.app.mapreduce.am.staging-dir} + * | -- $username + * | -- .staging rwx------ + * | -- $jobID rwx-----x + * | -- application rwx----- + * | -- files rwx------ + * | -- libjars rwx------ + * | -- archives rwx------ + * | -- private rwx----- + * | -- files rwx------ + * | -- libjars rwx------ + * | -- archives rwx------ + * | -- public rwxr-xr-x + * | -- files rwxr-xr-x + * | -- libjars rwxr-xr-x + * | -- archives rwxr-xr-x + * Job jar will be directly under one of {application, private, public} + * based on visibility configuration + */ // Create the submission directory for the MapReduce job. submitJobDir = jtFs.makeQualified(submitJobDir); submitJobDir = new Path(submitJobDir.toUri().getPath()); FsPermission mapredSysPerms = - new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION); + new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION_WORLD_EXCUTABLE); mkdirs(jtFs, submitJobDir, mapredSysPerms); + for (MRResourceVisibility visibility : MRResourceVisibility.values()) { + FsPermission permission = visibility.getDirPermission(); + Path visibilityPath = new Path(submitJobDir, visibility.getDirName()); + mkdirs(jtFs, visibilityPath, permission); + for (MRResourceType resourceType : MRResourceType.values()) { + if (resourceType.getResourceParentDirName().isEmpty()) { + continue; + } + Path resourceParentPath = + new Path(visibilityPath, resourceType.getResourceParentDirName()); + mkdirs(jtFs, resourceParentPath, permission); + } + } + // Clear stat cache as the size and timestamp are not correct + statCache.clear(); + } + + private void uploadResourcesInternal(Job job, Path submitJobDir) + throws IOException { + Configuration conf = job.getConfiguration(); + if (!(conf.getBoolean(Job.USED_GENERIC_PARSER, false))) { + LOG.warn("Hadoop command-line option parsing not performed. " + + "Implement the Tool interface and execute your application " + + "with ToolRunner to remedy this."); + } + + initJobAndResourceDirs(job, submitJobDir); if (!conf.getBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED, MRJobConfig.DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED)) { disableErasureCodingForPath(submitJobDir); } - // Get the resources that have been added via command line arguments in the - // GenericOptionsParser (i.e. files, libjars, archives). - Collection files = conf.getStringCollection("tmpfiles"); - Collection libjars = conf.getStringCollection("tmpjars"); - Collection archives = conf.getStringCollection("tmparchives"); - String jobJar = job.getJar(); - - // Merge resources that have been programmatically specified for the shared - // cache via the Job API. - files.addAll(conf.getStringCollection(MRJobConfig.FILES_FOR_SHARED_CACHE)); - libjars.addAll(conf.getStringCollection( - MRJobConfig.FILES_FOR_CLASSPATH_AND_SHARED_CACHE)); - archives.addAll(conf - .getStringCollection(MRJobConfig.ARCHIVES_FOR_SHARED_CACHE)); - - - Map statCache = new HashMap(); - checkLocalizationLimits(conf, files, libjars, archives, jobJar, statCache); - - Map fileSCUploadPolicies = - new LinkedHashMap(); - Map archiveSCUploadPolicies = - new LinkedHashMap(); - - uploadFiles(job, files, submitJobDir, mapredSysPerms, replication, - fileSCUploadPolicies, statCache); - uploadLibJars(job, libjars, submitJobDir, mapredSysPerms, replication, - fileSCUploadPolicies, statCache); - uploadArchives(job, archives, submitJobDir, mapredSysPerms, replication, - archiveSCUploadPolicies, statCache); - uploadJobJar(job, jobJar, submitJobDir, replication, statCache); - addLog4jToDistributedCache(job, submitJobDir); + // The method serve three purposes + // 1. Get the resources that have been added via command line arguments + // in the GenericOptionsParser (i.e. files, libjars, archives). + // 2. Merge resources that have been programmatically specified + // for the shared cache via the Job API. + // 3. Handle files which are added to via + // {@link org.apache.hadoop.mapreduce.Job#addCacheFile}. + // We need to determine file sizes, visibilities, and modification time. + // Note the API is deprecated, people shouldn't use it any more. + // We need to handle it for compatibility reason. + // We should think of fixing it in the future + + // Handle case 3 first as it will clear and reset flags used + // by following logic. Note the resource visibility is determined + // by resource's visibility here. + ClientDistributedCacheManager.determineTimestampsAndCacheVisibilities( + conf, statCache); + + List filesList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.FILE, conf); + List libjarsList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.LIBJAR, conf); + List archivesList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.ARCHIVE, conf); + MRResource jobJarResource = MRResourceUtil.getJobJar(conf); + + checkLocalizationLimits( + conf, filesList, libjarsList, archivesList, jobJarResource); + checkVisibilitySettings( + conf, filesList, libjarsList, archivesList, jobJarResource); + + short replication = + (short) conf.getInt(Job.SUBMIT_REPLICATION, + Job.DEFAULT_SUBMIT_REPLICATION); // Note, we do not consider resources in the distributed cache for the // shared cache at this time. Only resources specified via the // GenericOptionsParser or the jobjar. - Job.setFileSharedCacheUploadPolicies(conf, fileSCUploadPolicies); - Job.setArchiveSharedCacheUploadPolicies(conf, archiveSCUploadPolicies); + uploadJobJar(job, submitJobDir, jobJarResource, replication); + uploadFiles(job, submitJobDir, filesList, replication); + uploadLibJars(job, submitJobDir, libjarsList, replication); + uploadArchives(job, submitJobDir, archivesList, replication); + addLog4jToDistributedCache(job, submitJobDir); - // set the timestamps of the archives and files - // set the public/private visibility of the archives and files - ClientDistributedCacheManager.determineTimestampsAndCacheVisibilities(conf, - statCache); // get DelegationToken for cached file ClientDistributedCacheManager.getDelegationTokens(conf, job.getCredentials()); } - @VisibleForTesting - void uploadFiles(Job job, Collection files, - Path submitJobDir, FsPermission mapredSysPerms, short submitReplication, - Map fileSCUploadPolicies, Map statCache) + /** + * + * @param job + * @param submitJobDir + * @param resourceList + * @param submitReplication + * @return pair + * @throws IOException + */ + private List checkAndUploadMRResources( + Job job, Path submitJobDir, + List resourceList, short submitReplication) throws IOException { - Configuration conf = job.getConfiguration(); - Path filesDir = JobSubmissionFiles.getJobDistCacheFiles(submitJobDir); - if (!files.isEmpty()) { - mkdirs(jtFs, filesDir, mapredSysPerms); - for (String tmpFile : files) { - URI tmpURI = null; - try { - tmpURI = new URI(tmpFile); - } catch (URISyntaxException e) { - throw new IllegalArgumentException("Error parsing files argument." - + " Argument must be a valid URI: " + tmpFile, e); - } - Path tmp = new Path(tmpURI); - URI newURI = null; - boolean uploadToSharedCache = false; - if (scConfig.isSharedCacheFilesEnabled()) { - newURI = useSharedCache(tmpURI, tmp.getName(), statCache, conf, true); - if (newURI == null) { - uploadToSharedCache = true; - } - } - - if (newURI == null) { - Path newPath = - copyRemoteFiles(filesDir, tmp, conf, submitReplication); - try { - newURI = getPathURI(newPath, tmpURI.getFragment()); - } catch (URISyntaxException ue) { - // should not throw a uri exception - throw new IOException( - "Failed to create a URI (URISyntaxException) for the" - + " remote path " + newPath - + ". This was based on the files parameter: " + tmpFile, - ue); - } - } - - job.addCacheFile(newURI); - if (scConfig.isSharedCacheFilesEnabled()) { - fileSCUploadPolicies.put(newURI.toString(), uploadToSharedCache); - } - } + List mrResourceInfos = new ArrayList<>(); + for (MRResource resource : resourceList) { + mrResourceInfos.add( + checkAndUploadMRResource( + job, submitJobDir, resource, submitReplication)); } + return mrResourceInfos; } - // Suppress warning for use of DistributedCache (it is everywhere). - @SuppressWarnings("deprecation") + /** + * + * @param job + * @param submitJobDir + * @param resource + * @param submitReplication + * @return pair + * @throws IOException + */ @VisibleForTesting - void uploadLibJars(Job job, Collection libjars, Path submitJobDir, - FsPermission mapredSysPerms, short submitReplication, - Map fileSCUploadPolicies, Map statCache) - throws IOException { + MRResourceInfo checkAndUploadMRResource( + Job job, Path submitJobDir, + MRResource resource, short submitReplication) throws IOException { Configuration conf = job.getConfiguration(); - Path libjarsDir = JobSubmissionFiles.getJobDistCacheLibjars(submitJobDir); - if (!libjars.isEmpty()) { - mkdirs(jtFs, libjarsDir, mapredSysPerms); - Collection libjarURIs = new LinkedList<>(); - boolean foundFragment = false; - for (String tmpjars : libjars) { - URI tmpURI = null; - try { - tmpURI = new URI(tmpjars); - } catch (URISyntaxException e) { - throw new IllegalArgumentException("Error parsing libjars argument." - + " Argument must be a valid URI: " + tmpjars, e); - } - Path tmp = new Path(tmpURI); - URI newURI = null; - boolean uploadToSharedCache = false; - boolean fromSharedCache = false; - if (scConfig.isSharedCacheLibjarsEnabled()) { - newURI = useSharedCache(tmpURI, tmp.getName(), statCache, conf, true); - if (newURI == null) { - uploadToSharedCache = true; - } else { - fromSharedCache = true; - } - } - - if (newURI == null) { - Path newPath = - copyRemoteFiles(libjarsDir, tmp, conf, submitReplication); - try { - newURI = getPathURI(newPath, tmpURI.getFragment()); - } catch (URISyntaxException ue) { - // should not throw a uri exception - throw new IOException( - "Failed to create a URI (URISyntaxException) for the" - + " remote path " + newPath - + ". This was based on the libjar parameter: " + tmpjars, - ue); - } - } - - if (!foundFragment) { - // We do not count shared cache paths containing fragments as a - // "foundFragment." This is because these resources are not in the - // staging directory and will be added to the distributed cache - // separately. - foundFragment = (newURI.getFragment() != null) && !fromSharedCache; - } - DistributedCache.addFileToClassPath(new Path(newURI.getPath()), conf, - jtFs, false); - if (fromSharedCache) { - // We simply add this URI to the distributed cache. It will not come - // from the staging directory (it is in the shared cache), so we - // must add it to the cache regardless of the wildcard feature. - DistributedCache.addCacheFile(newURI, conf); - } else { - libjarURIs.add(newURI); - } + URI resourceUri = resource.getResourceUri(conf); + LOG.debug("Check and upload resource:" + resource.toString() + + ", resource uri:" + resourceUri); + // TODO: clean up the code of URI and path, now just copy the old logic + Path jobJarPath = new Path(resourceUri); + URI uriToUse = null; + boolean uriFromSharedCache = false; + if (isSharedCacheEnabled(resource)) { + uriToUse = useSharedCache(resourceUri, jobJarPath.getName(), conf, true); + uriFromSharedCache = uriToUse == null ? false : true; + } - if (scConfig.isSharedCacheLibjarsEnabled()) { - fileSCUploadPolicies.put(newURI.toString(), uploadToSharedCache); - } + if (uriToUse == null) { + Path submissionParentDir = + resource.getResourceSubmissionParentDir(submitJobDir); + // We must have a qualified path for the shared cache client. We can + // assume this is for the local filesystem + Path newPath = + copyRemoteFiles( + submissionParentDir, jobJarPath, conf, submitReplication); + LOG.debug("New path is:" + newPath + + ", submissionParentDir:" + submissionParentDir); + try { + uriToUse = getPathURI(newPath, resourceUri.getFragment()); + } catch (URISyntaxException ue) { + // should not throw a uri exception + throw new IOException( + "Failed to create a URI (URISyntaxException) for the" + + " remote path " + newPath + + ". This was based on the files parameter: " + resource, + ue); } + } + return new MRResourceInfo(resource, uriToUse, uriFromSharedCache); + } - if (useWildcard && !foundFragment) { - // Add the whole directory to the cache using a wild card - Path libJarsDirWildcard = - jtFs.makeQualified(new Path(libjarsDir, DistributedCache.WILDCARD)); - DistributedCache.addCacheFile(libJarsDirWildcard.toUri(), conf); - } else { - for (URI uri : libjarURIs) { - DistributedCache.addCacheFile(uri, conf); - } - } + public boolean isSharedCacheEnabled(MRResource resource) { + return isSharedCacheEnabled(scConfig, resource); + } + + public static boolean isSharedCacheEnabled( + SharedCacheConfig scConf, MRResource resource) { + if (resource.getResourceVisibility() != MRResourceVisibility.PUBLIC) { + // Only public resource is supported for the moment. + return false; + } + switch (resource.getResourceType()) { + case JOBJAR: + return scConf.isSharedCacheJobjarEnabled(); + case FILE: + return scConf.isSharedCacheFilesEnabled(); + case LIBJAR: + return scConf.isSharedCacheLibjarsEnabled(); + case ARCHIVE: + return scConf.isSharedCacheArchivesEnabled(); + default: + return false; } } @VisibleForTesting - void uploadArchives(Job job, Collection archives, - Path submitJobDir, FsPermission mapredSysPerms, short submitReplication, - Map archiveSCUploadPolicies, - Map statCache) throws IOException { + void uploadFiles( + Job job, Path submitJobDir, + List resourceList, short submitReplication) + throws IOException { + if (resourceList.isEmpty()) { + return; + } + Map uploadPolicyMap = new HashMap<>(); Configuration conf = job.getConfiguration(); - Path archivesDir = JobSubmissionFiles.getJobDistCacheArchives(submitJobDir); - if (!archives.isEmpty()) { - mkdirs(jtFs, archivesDir, mapredSysPerms); - for (String tmpArchives : archives) { - URI tmpURI; - try { - tmpURI = new URI(tmpArchives); - } catch (URISyntaxException e) { - throw new IllegalArgumentException("Error parsing archives argument." - + " Argument must be a valid URI: " + tmpArchives, e); - } - Path tmp = new Path(tmpURI); - URI newURI = null; - boolean uploadToSharedCache = false; - if (scConfig.isSharedCacheArchivesEnabled()) { - newURI = useSharedCache(tmpURI, tmp.getName(), statCache, conf, true); - if (newURI == null) { - uploadToSharedCache = true; - } - } - - if (newURI == null) { - Path newPath = - copyRemoteFiles(archivesDir, tmp, conf, submitReplication); - try { - newURI = getPathURI(newPath, tmpURI.getFragment()); - } catch (URISyntaxException ue) { - // should not throw a uri exception - throw new IOException( - "Failed to create a URI (URISyntaxException) for the" - + " remote path " + newPath - + ". This was based on the archive parameter: " - + tmpArchives, - ue); - } - } + for (MRResource resource : resourceList) { + MRResourceInfo resourceInfo = + checkAndUploadMRResource( + job, submitJobDir, resource, submitReplication); + addFileToJobCache(job, resourceInfo); + uploadPolicyMap.put( + resourceInfo.uriToUseInJob.toString(), + resourceInfo.getUploadPolicy(scConfig)); + } + job.appendFileSharedCacheUploadPolicies(conf, uploadPolicyMap); + } - job.addCacheArchive(newURI); - if (scConfig.isSharedCacheArchivesEnabled()) { - archiveSCUploadPolicies.put(newURI.toString(), uploadToSharedCache); - } - } + @VisibleForTesting + void uploadArchives( + Job job, Path submitJobDir, + List resourceList, short submitReplication) + throws IOException { + if (resourceList.isEmpty()) { + return; + } + Map uploadPolicyMap = new HashMap<>(); + Configuration conf = job.getConfiguration(); + for (MRResource resource : resourceList) { + MRResourceInfo resourceInfo = + checkAndUploadMRResource( + job, submitJobDir, resource, submitReplication); + DistributedCache.addCacheArchiveWithMeta( + resourceInfo.uriToUseInJob, + resourceInfo.getYarnResourceVisibility(), + resourceInfo.getUriToUseFileSize(conf, statCache), + resourceInfo.getUriToUseTimestamp(conf, statCache), + conf); + uploadPolicyMap.put( + resourceInfo.uriToUseInJob.toString(), + resourceInfo.getUploadPolicy(scConfig)); } + job.setArchiveSharedCacheUploadPolicies(conf, uploadPolicyMap); } @VisibleForTesting - void uploadJobJar(Job job, String jobJar, Path submitJobDir, - short submitReplication, Map statCache) + void uploadLibJars( + Job job, Path submitJobDir, + List resourceList, short submitReplication) throws IOException { + if (resourceList.isEmpty()) { + return; + } Configuration conf = job.getConfiguration(); - if (jobJar != null) { // copy jar to JobTracker's fs - // use jar name if job is not named. - if ("".equals(job.getJobName())) { - job.setJobName(new Path(jobJar).getName()); + + boolean foundFragment = false; + List notFromSharedCacheJars = new LinkedList<>(); + Map uploadPolicies = new HashMap<>(); + for (MRResource resource : resourceList) { + MRResourceInfo resourceInfo = + checkAndUploadMRResource( + job, submitJobDir, resource, submitReplication); + URI resourceUri = resourceInfo.uriToUseInJob; + if (!foundFragment) { + // We do not count shared cache paths containing fragments as a + // "foundFragment." This is because these resources are not in the + // staging directory and will be added to the distributed cache + // separately. + foundFragment = (resourceInfo.uriToUseInJob.getFragment() != null) + && !resourceInfo.isFromShareCache; } - Path jobJarPath = new Path(jobJar); - URI jobJarURI = jobJarPath.toUri(); - Path newJarPath = null; - boolean uploadToSharedCache = false; - if (jobJarURI.getScheme() == null || - jobJarURI.getScheme().equals("file")) { - // job jar is on the local file system - if (scConfig.isSharedCacheJobjarEnabled()) { - // We must have a qualified path for the shared cache client. We can - // assume this is for the local filesystem - jobJarPath = FileSystem.getLocal(conf).makeQualified(jobJarPath); - // Don't add a resource name here because the resource name (i.e. - // job.jar directory symlink) will always be hard coded to job.jar for - // the job.jar - URI newURI = - useSharedCache(jobJarPath.toUri(), null, statCache, conf, false); - if (newURI == null) { - uploadToSharedCache = true; - } else { - newJarPath = stringToPath(newURI.toString()); - // The job jar is coming from the shared cache (i.e. a public - // place), so we want the job.jar to have a public visibility. - conf.setBoolean(MRJobConfig.JOBJAR_VISIBILITY, true); - } - } - if (newJarPath == null) { - newJarPath = JobSubmissionFiles.getJobJar(submitJobDir); - copyJar(jobJarPath, newJarPath, submitReplication); - } + DistributedCache.addFileToClassPath( + new Path(resourceUri.getPath()), conf, jtFs, false); + if (resourceInfo.isFromShareCache) { + // We simply add this URI to the distributed cache. It will not come + // from the staging directory (it is in the shared cache), so we + // must add it to the cache regardless of the wildcard feature. + addFileToJobCache(job, resourceInfo); } else { - // job jar is in a remote file system - if (scConfig.isSharedCacheJobjarEnabled()) { - // Don't add a resource name here because the resource name (i.e. - // job.jar directory symlink) will always be hard coded to job.jar for - // the job.jar - URI newURI = useSharedCache(jobJarURI, null, statCache, conf, false); - if (newURI == null) { - uploadToSharedCache = true; - newJarPath = jobJarPath; - } else { - newJarPath = stringToPath(newURI.toString()); - // The job jar is coming from the shared cache (i.e. a public - // place), so we want the job.jar to have a public visibility. - conf.setBoolean(MRJobConfig.JOBJAR_VISIBILITY, true); - } - } else { - // we don't need to upload the jobjar to the staging directory because - // it is already in an accessible place - newJarPath = jobJarPath; - } + notFromSharedCacheJars.add(resourceInfo); } - job.setJar(newJarPath.toString()); - if (scConfig.isSharedCacheJobjarEnabled()) { - conf.setBoolean(MRJobConfig.JOBJAR_SHARED_CACHE_UPLOAD_POLICY, - uploadToSharedCache); + uploadPolicies.put( + resourceInfo.uriToUseInJob.toString(), + resourceInfo.getUploadPolicy(scConfig)); + } + + if (useWildcard && !foundFragment) { + // Add the whole directory to the cache using a wild card + // addLibJarDirs(job, submitJobDir); + + // TODO: Share cache manger got a problem while dealing with wildcards. + // So will add jar file by file here. + for (MRResourceInfo resourceInfo : notFromSharedCacheJars) { + addFileToJobCache(job, resourceInfo); } } else { + for (MRResourceInfo resourceInfo : notFromSharedCacheJars) { + addFileToJobCache(job, resourceInfo); + } + } + job.appendFileSharedCacheUploadPolicies(conf, uploadPolicies); + } + + @VisibleForTesting + void addLibJarDirs(Job job, Path submitJobDir) throws IOException { + Configuration conf = job.getConfiguration(); + Map uploadPolicies = new HashMap<>(); + for (MRResourceVisibility visibility : MRResourceVisibility.values()) { + Path libJarsDirWildcard = + jtFs.makeQualified( + new Path( + JobSubmissionFiles.getJobDistCacheLibjarsPath( + submitJobDir, visibility), DistributedCache.WILDCARD)); + MRResource resource = new MRResource( + MRResourceType.LIBJAR, libJarsDirWildcard.toString(), visibility); + MRResourceInfo resourceInfo = new MRResourceInfo( + resource, libJarsDirWildcard.toUri(), false); + addFileToJobCache(job, resourceInfo); + uploadPolicies.put(libJarsDirWildcard.toString(), + resourceInfo.getUploadPolicy(scConfig)); + LOG.info("Upload policy for " + libJarsDirWildcard.toString() + " is:" + + resourceInfo.getUploadPolicy(scConfig)); + } + job.appendFileSharedCacheUploadPolicies(conf, uploadPolicies); + } + + private void addFileToJobCache(Job job, MRResourceInfo resourceInfo) + throws IOException { + DistributedCache.addCacheFileWithMeta( + resourceInfo.uriToUseInJob, + resourceInfo.getYarnResourceVisibility(), + resourceInfo.getUriToUseTimestamp(job.getConfiguration(), statCache), + resourceInfo.getUriToUseFileSize(job.getConfiguration(), statCache), + job.getConfiguration()); + } + + void uploadJobJar( + Job job, Path submitJobDir, + MRResource jobJarResource, short submitReplication) + throws IOException { + if (jobJarResource == null || jobJarResource.getResourcePathStr() == null) { LOG.warn("No job jar file set. User classes may not be found. " + "See Job or Job#setJar(String)."); + return; + } + if (job.getJobName().trim().isEmpty()) { + // use jar name if job is not named. + job.setJobName(new Path(jobJarResource.getResourcePathStr()).getName()); + } + MRResourceInfo resourceInfo + = checkAndUploadMRResource( + job, submitJobDir, jobJarResource, submitReplication); + // Remove fragment in uri to keep the old behavior + resourceInfo.removeFragmentInUri(); + Configuration conf = job.getConfiguration(); + LOG.debug("Jar jar:" + jobJarResource.toString() + + ", URIToUse:" + resourceInfo.uriToUseInJob + + ", isFromSharedcache:" + resourceInfo.isFromShareCache); + // {@link MRJobConfig.CACHE_JOBJAR_VISIBILITY} will be used + // after 2.9 to replace {@link MRJobConfig.JOBJAR_VISIBILITY}. + // JOBJAR_VISIBILITY will be user facing only. + conf.set(MRJobConfig.CACHE_JOBJAR_VISIBILITY, + jobJarResource.getResourceVisibility().name()); + conf.setBoolean(MRJobConfig.JOBJAR_SHARED_CACHE_UPLOAD_POLICY, + determineResourceUploadPolicy( + scConfig, jobJarResource, resourceInfo.isFromShareCache)); + job.setJar(resourceInfo.uriToUseInJob.toString()); + } + + /** + * Verify the visibility settings to avoid permission violation. + * For the moment, we only check to make sure resource with + * public visibilities are public accessible. + */ + @VisibleForTesting + void checkVisibilitySettings( + Configuration conf, List files, List libjars, + List archives, MRResource jobJar) throws IOException { + if (jobJar != null) { + checkResourceVisibilitySettings(conf, jobJar); + } + checkResourceVisibilitySettings(conf, files); + checkResourceVisibilitySettings(conf, libjars); + checkResourceVisibilitySettings(conf, archives); + } + + private void checkResourceVisibilitySettings( + Configuration conf, List resourceList) throws IOException { + for (MRResource resource : resourceList) { + checkResourceVisibilitySettings(conf, resource); + } + } + + private void checkResourceVisibilitySettings( + Configuration conf, MRResource resource) throws IOException { + if (resource.getResourceVisibility() != MRResourceVisibility.PUBLIC) { + // We will only check public resources for the moment. + return; } + URI uri = resource.getResourceUri(conf); + if (uri == null || uri.toString().isEmpty()) { + throw new IOException(resource.toString() + + " is illegal, path shouldn't be null or empty"); + } + // We won't enforce local files to be public accessible to use share cache. + // For files on HDFS, we have to check whether it's public accessible as + // node manager will enforce it. + if (!isLocalFile(uri) && !ClientDistributedCacheManager.isPublic( + conf, uri, statCache)) { + throw new IOException(resource.toString() + + " is not public accessible and can't set to be public"); + } + } + + private static boolean isLocalFile(URI uri) { + if (uri.getScheme() == null || uri.getScheme().isEmpty() + || uri.getScheme().startsWith("file")) { + return true; + } + return false; } /** @@ -487,47 +676,65 @@ void uploadJobJar(Job job, String jobJar, Path submitJobDir, * localization limits. We count all resources towards these limits regardless * of where they are coming from (i.e. local, distributed cache, or shared * cache). + * @param + * @return return */ @VisibleForTesting - void checkLocalizationLimits(Configuration conf, Collection files, - Collection libjars, Collection archives, String jobJar, - Map statCache) throws IOException { - + void checkLocalizationLimits( + Configuration conf, List files, List libjars, + List archives, MRResource jobJar) + throws IOException { + Map statsCache; LimitChecker limitChecker = new LimitChecker(conf); if (!limitChecker.hasLimits()) { // there are no limits set, so we are done. return; } + checkExistingResourceLimits(conf, limitChecker); + if (jobJar != null) { + // Keep the behavior compatible with old logic + exploreResource(conf, jobJar, limitChecker); + } + exploreResources(conf, files, limitChecker); + exploreResources(conf, libjars, limitChecker); + exploreResources(conf, archives, limitChecker); + } + private void checkExistingResourceLimits( + Configuration conf, LimitChecker limitChecker) + throws IOException { // Get the files and archives that are already in the distributed cache Collection dcFiles = conf.getStringCollection(MRJobConfig.CACHE_FILES); + explorePaths(conf, dcFiles, limitChecker); Collection dcArchives = conf.getStringCollection(MRJobConfig.CACHE_ARCHIVES); + explorePaths(conf, dcArchives, limitChecker); + } - for (String uri : dcFiles) { - explorePath(conf, stringToPath(uri), limitChecker, statCache); - } - - for (String uri : dcArchives) { - explorePath(conf, stringToPath(uri), limitChecker, statCache); - } - - for (String uri : files) { - explorePath(conf, stringToPath(uri), limitChecker, statCache); - } - - for (String uri : libjars) { - explorePath(conf, stringToPath(uri), limitChecker, statCache); + private void explorePaths( + Configuration conf, Collection paths, LimitChecker limitChecker) + throws IOException { + for (String path : paths) { + explorePath(conf, stringToPath(path), limitChecker); } + } - for (String uri : archives) { - explorePath(conf, stringToPath(uri), limitChecker, statCache); + private void exploreResources( + Configuration conf, + List resourceList, + LimitChecker limitChecker) + throws IOException{ + for (MRResource resource : resourceList) { + exploreResource(conf, resource, limitChecker); } + } - if (jobJar != null) { - explorePath(conf, stringToPath(jobJar), limitChecker, statCache); - } + private void exploreResource( + Configuration jobConf, MRResource resource, LimitChecker limitChecker) + throws IOException { + Path path = stringToPath(resource.getResourcePathStr()); + explorePath(jobConf, path, limitChecker); } /** @@ -615,7 +822,7 @@ private void addFile(Path p, long fileSizeBytes) throws IOException { * directory structure. */ private void explorePath(Configuration job, Path p, - LimitChecker limitChecker, Map statCache) + LimitChecker limitChecker) throws IOException { Path pathWithScheme = p; if (!pathWithScheme.toUri().isAbsolute()) { @@ -624,12 +831,12 @@ private void explorePath(Configuration job, Path p, FileSystem localFs = FileSystem.getLocal(job); pathWithScheme = localFs.makeQualified(p); } - FileStatus status = getFileStatus(statCache, job, pathWithScheme); + FileStatus status = getFileStatus(job, pathWithScheme); if (status.isDirectory()) { FileStatus[] statusArray = pathWithScheme.getFileSystem(job).listStatus(pathWithScheme); for (FileStatus s : statusArray) { - explorePath(job, s.getPath(), limitChecker, statCache); + explorePath(job, s.getPath(), limitChecker); } } else { limitChecker.addFile(pathWithScheme, status.getLen()); @@ -637,13 +844,16 @@ private void explorePath(Configuration job, Path p, } @VisibleForTesting - FileStatus getFileStatus(Map statCache, - Configuration job, Path p) throws IOException { + FileStatus getFileStatus(Configuration job, Path p) throws IOException { URI u = p.toUri(); FileStatus status = statCache.get(u); if (status == null) { status = p.getFileSystem(job).getFileStatus(p); - statCache.put(u, status); + if (!status.isDirectory()) { + // Buffer dir could be tricky as we don't change the cache + // while add files to the dir. So disable it here. + statCache.put(u, status); + } } return status; } @@ -687,13 +897,13 @@ Path copyRemoteFiles(Path parentDir, Path originalPath, * Checksum a local resource file and call use for that resource with the scm. */ private URI useSharedCache(URI sourceFile, String resourceName, - Map statCache, Configuration conf, boolean honorFragment) + Configuration conf, boolean honorFragment) throws IOException { if (scClient == null) { return null; } Path filePath = new Path(sourceFile); - if (getFileStatus(statCache, conf, filePath).isDirectory()) { + if (getFileStatus(conf, filePath).isDirectory()) { LOG.warn("Shared cache does not support directories" + " (see YARN-6097)." + " Will not upload " + filePath + " to the shared cache."); @@ -757,24 +967,37 @@ private URI useSharedCache(URI sourceFile, String resourceName, } } - @VisibleForTesting - void copyJar(Path originalJarPath, Path submitJarFile, - short replication) throws IOException { - jtFs.copyFromLocalFile(originalJarPath, submitJarFile); - jtFs.setReplication(submitJarFile, replication); - jtFs.setPermission(submitJarFile, new FsPermission( - JobSubmissionFiles.JOB_FILE_PERMISSION)); + private MRResource getLog4jResource(Configuration conf) { + String log4jPropertyFile = + conf.get(MRJobConfig.MAPREDUCE_JOB_LOG4J_PROPERTIES_FILE, ""); + if (log4jPropertyFile.isEmpty()) { + return null; + } + MRResourceVisibility vis = + MRResourceVisibility.getVisibility( + conf.get( + MRJobConfig.MAPREDUCE_JOB_LOG4J_PROPERTIES_FILE_VISIBILITY)); + if (vis == null) { + vis = MRJobConfig.MAPREDUCE_JOB_LOG4J_PROPERTIES_FILE_VISIBILITY_DEFAULT; + } + return new MRResource(MRResourceType.FILE, log4jPropertyFile, vis); } private void addLog4jToDistributedCache(Job job, Path jobSubmitDir) throws IOException { Configuration conf = job.getConfiguration(); - String log4jPropertyFile = - conf.get(MRJobConfig.MAPREDUCE_JOB_LOG4J_PROPERTIES_FILE, ""); - if (!log4jPropertyFile.isEmpty()) { - short replication = (short) conf.getInt(Job.SUBMIT_REPLICATION, 10); - copyLog4jPropertyFile(job, jobSubmitDir, replication); + MRResource log4jPropertyFile = getLog4jResource(conf); + if (log4jPropertyFile == null) { + return; } + short replication = (short) conf.getInt(Job.SUBMIT_REPLICATION, 10); + MRResourceInfo resourceInfo = + checkAndUploadMRResource( + job, jobSubmitDir, log4jPropertyFile, replication); + addFileToJobCache(job, resourceInfo); + FileSystem fs = FileSystem.get(resourceInfo.uriToUseInJob, conf); + DistributedCache.addFileToClassPath( + new Path(resourceInfo.uriToUseInJob), conf, fs, false); } private URI getPathURI(Path destPath, String fragment) @@ -825,7 +1048,7 @@ private void copyLog4jPropertyFile(Job job, Path submitJobDir, Path tmp = new Path(tmpURI); Path newPath = copyRemoteFiles(fileDir, tmp, conf, replication); DistributedCache.addFileToClassPath(new Path(newPath.toUri().getPath()), - conf); + conf, FileSystem.get(newPath.toUri(), conf), false); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java index 303aa13..c0d5afd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java @@ -20,6 +20,8 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.FileStatus; @@ -45,10 +47,13 @@ // job submission directory is private! final public static FsPermission JOB_DIR_PERMISSION = FsPermission.createImmutable((short) 0700); // rwx------ + + final public static FsPermission JOB_DIR_PERMISSION_WORLD_EXCUTABLE = + FsPermission.createImmutable((short) 0701); // rwx-----x //job files are world-wide readable and owner writable final public static FsPermission JOB_FILE_PERMISSION = FsPermission.createImmutable((short) 0644); // rw-r--r-- - + public static Path getJobSplitFile(Path jobSubmissionDir) { return new Path(jobSubmissionDir, "job.split"); } @@ -63,14 +68,7 @@ public static Path getJobSplitMetaFile(Path jobSubmissionDir) { public static Path getJobConfPath(Path jobSubmitDir) { return new Path(jobSubmitDir, "job.xml"); } - - /** - * Get the job jar path. - */ - public static Path getJobJar(Path jobSubmitDir) { - return new Path(jobSubmitDir, "job.jar"); - } - + /** * Get the job distributed cache files path. * @param jobSubmitDir @@ -78,7 +76,7 @@ public static Path getJobJar(Path jobSubmitDir) { public static Path getJobDistCacheFiles(Path jobSubmitDir) { return new Path(jobSubmitDir, "files"); } - + /** * Get the job distributed cache path for log4j properties. * @param jobSubmitDir @@ -101,6 +99,21 @@ public static Path getJobDistCacheLibjars(Path jobSubmitDir) { return new Path(jobSubmitDir, "libjars"); } + public static List getJobDistCacheLibjarsPaths(Path jobSubmitDir) { + List pathList = new ArrayList<>(); + for (MRResourceVisibility visibility : MRResourceVisibility.values()) { + pathList.add(getJobDistCacheLibjarsPath(jobSubmitDir, visibility)); + } + return pathList; + } + + public static Path getJobDistCacheLibjarsPath( + Path jobSubmitDir, MRResourceVisibility visibility) { + return new Path( + jobSubmitDir, visibility.getDirName() + "/" + + MRResourceType.LIBJAR.getResourceParentDirName()); + } + /** * Initializes the staging directory and returns the path. It also * keeps track of all necessary ownership and permissions @@ -132,6 +145,7 @@ public static Path getStagingDir(Cluster cluster, Configuration conf, Path stagingArea = cluster.getStagingAreaDir(); FileSystem fs = stagingArea.getFileSystem(conf); UserGroupInformation currentUser = realUser.getCurrentUser(); + FsPermission stagingDirPermission = getStagingDirPermission(conf); try { FileStatus fsStatus = fs.getFileStatus(stagingArea); String fileOwner = fsStatus.getOwner(); @@ -152,15 +166,24 @@ public static Path getStagingDir(Cluster cluster, Configuration conf, throw new IOException(errorMessage); } } - if (!fsStatus.getPermission().equals(JOB_DIR_PERMISSION)) { + if (!fsStatus.getPermission().equals(stagingDirPermission)) { LOG.info("Permissions on staging directory " + stagingArea + " are " + "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " + - "to correct value " + JOB_DIR_PERMISSION); - fs.setPermission(stagingArea, JOB_DIR_PERMISSION); + "to correct value " + stagingDirPermission); + fs.setPermission(stagingArea, stagingDirPermission); } } catch (FileNotFoundException e) { - fs.mkdirs(stagingArea, new FsPermission(JOB_DIR_PERMISSION)); + fs.mkdirs(stagingArea, new FsPermission(stagingDirPermission)); } return stagingArea; } + + public static FsPermission getStagingDirPermission(Configuration conf) { + String permStr = conf.get(MRJobConfig.MR_JOB_STAGING_DIR_PERMISSION); + if (permStr == null) { + return FsPermission.createImmutable( + MRJobConfig.DEFAULT_MR_JOB_STAGING_DIR_PERMISSION); + } + return FsPermission.createImmutable(Short.decode(permStr)); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java index fb34698..5eed711 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java @@ -211,14 +211,56 @@ public static final String CACHE_ARCHIVES_VISIBILITIES = "mapreduce.job.cache.archives.visibilities"; /** + * The is a generated parameter. MR client will set it based on + * JOBJAR_VISIBILITY. It will be used to determine resource visibility as yarn + * resource in shared/distributed cache. It won't be set when the client + * is older than 2.9. + */ + String CACHE_JOBJAR_VISIBILITY = + "mapreduce.job.cache.jobjar.visibility"; + + /** * This parameter controls the visibility of the localized job jar on the node - * manager. If set to true, the visibility will be set to - * LocalResourceVisibility.PUBLIC. If set to false, the visibility will be set - * to LocalResourceVisibility.APPLICATION. This is a generated parameter and - * should not be set manually via config files. + * manager. It should be one of + * #MRResourceVisibility{PUBLIC, PRIVATE, APPLICATION}. This is an user facing + * config item. */ String JOBJAR_VISIBILITY = "mapreduce.job.jobjar.visibility"; - boolean JOBJAR_VISIBILITY_DEFAULT = false; + MRResourceVisibility JOBJAR_VISIBILITY_DEFAULT = + MRResourceVisibility.APPLICATION; + + /** + * This parameter controls the visibility of the localized lib jars on + * the node manager. It should be one of + * #MRResourceVisibility{PUBLIC, PRIVATE, APPLICATION} + * Note that it will only work for resources whose visibility is not + * set implicitly. + */ + String LIBJARS_VISIBILITY = "mapreduce.job.libjars.visibility"; + MRResourceVisibility LIBJARS_VISIBILITY_DEFAULT = + MRResourceVisibility.APPLICATION; + + /** + * This parameter controls the visibility of the localized files on the node + * manager. It should be one of + * #MRResourceVisibility{PUBLIC, PRIVATE, APPLICATION} + * Note that it will only work for resources whose visibility is not + * set implicitly. + */ + String FILES_VISIBILITY = "mapreduce.job.files.visibility"; + MRResourceVisibility FILES_VISIBILITY_DEFAULT = + MRResourceVisibility.APPLICATION; + + /** + * This parameter controls the visibility of the localized archives on + * the node manager. It should be one of + * #MRResourceVisibility{PUBLIC, PRIVATE, APPLICATION} + * Note that it will only work for resources whose visibility is not set + * implicitly. + */ + String ARCHIEVS_VISIBILITY = "mapreduce.job.archives.visibility"; + MRResourceVisibility ARCHIEVES_VISIBILITY_DEFAULT = + MRResourceVisibility.APPLICATION; /** * This is a generated parameter and should not be set manually via config @@ -687,6 +729,31 @@ public static final String DEFAULT_MR_AM_STAGING_DIR = "/tmp/hadoop-yarn/staging"; + /** + * This flag is used to set the last dir name of users's staging path. + * A MR job's staging dir is defined in + * {\@link org.apache.hadoop.mapreduce.v2.util.MRApps#getStagingAreaDir}. + * ${MRJobConfig.MR_AM_STAGING_DIR}/${user}/${MR_USER_STAGING_DIR_NAME}; + * + * E.g. user1 launched two jobs: job-1234 and job-abcd. Their staging dir + * will be + * ${MRJobConfig.MR_AM_STAGING_DIR}/user1/${MR_USER_STAGING_DIR_NAME}/job-1234 + * ${MRJobConfig.MR_AM_STAGING_DIR}/user1/${MR_USER_STAGING_DIR_NAME}/job-abcd + */ + String MR_USER_STAGING_DIR_NAME + = "mapreduce.job.user.staging.dirname"; + String DEFAULT_MR_USER_STAGING_DIR_NAME + = ".staging"; + + /** + * Permission of MR job staging dir, default is 0700. + * Only numeric style are supported for the moment + */ + String MR_JOB_STAGING_DIR_PERMISSION = + "mapreduce.job.staging.dir.permission"; + short DEFAULT_MR_JOB_STAGING_DIR_PERMISSION = + (short) 0700; // rwx------ + /** The amount of memory the MR app master needs. * Kept for backward-compatibility, yarn.app.mapreduce.am.resource.memory is * the new preferred way to specify this @@ -1072,9 +1139,16 @@ public static final String MAPREDUCE_APPLICATION_CLASSPATH = "mapreduce.application.classpath"; - public static final String MAPREDUCE_JOB_LOG4J_PROPERTIES_FILE = + String MAPREDUCE_JOB_LOG4J_PROPERTIES_FILE = "mapreduce.job.log4j-properties-file"; + String MAPREDUCE_JOB_LOG4J_PROPERTIES_FILE_VISIBILITY + = "mapreduce.job.log4j-properties-file.visibility"; + + MRResourceVisibility + MAPREDUCE_JOB_LOG4J_PROPERTIES_FILE_VISIBILITY_DEFAULT + = MRResourceVisibility.APPLICATION; + /** * Path to MapReduce framework archive */ diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRResource.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRResource.java new file mode 100644 index 0000000..fcbae90 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRResource.java @@ -0,0 +1,110 @@ +/** + * 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.mapreduce; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; + +/** + * The class is used to represent a resource of MapReduce job. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class MRResource { + + private MRResourceType resourceType; + private MRResourceVisibility resourceVisibility; + private String resourceUriStr; + + public MRResource( + MRResourceType type, String path, MRResourceVisibility visibility) { + resourceType = type; + resourceUriStr = path; + resourceVisibility = visibility; + } + + public MRResourceType getResourceType() { + return resourceType; + } + + public void setResourceType(MRResourceType resourceType) { + this.resourceType = resourceType; + } + + public MRResourceVisibility getResourceVisibility() { + return resourceVisibility; + } + + public LocalResourceVisibility getYarnLocalResourceVisibility() { + return MRResourceVisibility.getYarnLocalResourceVisibility( + resourceVisibility); + } + + public void setResourceVisibility(MRResourceVisibility resourceVisibility) { + this.resourceVisibility = resourceVisibility; + } + + public String getResourcePathStr() { + return resourceUriStr; + } + + public void setResourceUriStr(String resourceUriStr) { + this.resourceUriStr = resourceUriStr; + } + + public Path getResourceSubmissionParentDir(Path jobSubmissionDir) { + if (resourceType.getResourceParentDirName().isEmpty()) { + return new Path(jobSubmissionDir, + resourceVisibility.getDirName()); + } else { + return new Path(jobSubmissionDir, + resourceVisibility.getDirName() + + "/" + resourceType.getResourceParentDirName()); + } + } + + public URI getResourceUri(Configuration conf) throws IOException { + Path path = new Path(resourceUriStr); + URI uri = path.toUri(); + if (uri == null + || uri.getScheme() == null + || uri.getScheme().equals("file")) { + path = FileSystem.getLocal(conf).makeQualified(path); + } + try { + return new URI(path.toString()); + } catch (URISyntaxException e) { + throw new IllegalArgumentException("Error parsing files argument." + + " Argument must be a valid URI: " + toString(), e); + } + } + + @Override + public String toString() { + return "MRResource resourceType:" + resourceType + ", resourceVisibility:" + + resourceVisibility + ", resource path:" + resourceUriStr; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRResourceType.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRResourceType.java new file mode 100644 index 0000000..5dd4e92 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRResourceType.java @@ -0,0 +1,44 @@ +/** + * 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.mapreduce; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The class defines the type of resource in MapReduce. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public enum MRResourceType { + JOBJAR(""), + LIBJAR("libjars"), + FILE("files"), + ARCHIVE("archives"); + + private String resourceParentDirName; + + MRResourceType(String dirName) { + resourceParentDirName = dirName; + } + + public String getResourceParentDirName() { + return resourceParentDirName; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRResourceVisibility.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRResourceVisibility.java new file mode 100644 index 0000000..cf05180 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRResourceVisibility.java @@ -0,0 +1,104 @@ +/** + * 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.mapreduce; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; + +/** + * This enum is used to represent visibility of MapReduce resource. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public enum MRResourceVisibility { + // DirPermision, FilePermission + PUBLIC(FsPermission.createImmutable((short) 0755), + FsPermission.createImmutable((short) 0755), "public"), + PRIVATE(FsPermission.createImmutable((short) 0750), + FsPermission.createImmutable((short) 0750), "private"), + APPLICATION(FsPermission.createImmutable((short) 0700), + FsPermission.createImmutable((short) 0700), "application"); + + private FsPermission dirPermission; + private FsPermission filePermission; + // The parent dir name for resources. + private String dirName; + + MRResourceVisibility( + FsPermission dirPerm, FsPermission filePerm, String dirNam) { + dirPermission = dirPerm; + filePermission = filePerm; + dirName = dirNam; + } + + public FsPermission getDirPermission() { + return dirPermission; + } + + public FsPermission getFilePermission() { + return filePermission; + } + + public String getDirName() { + return dirName; + } + + public static boolean contains(String visibility) { + if (visibility == null) { + return false; + } + visibility = visibility.toUpperCase(); + for (MRResourceVisibility v : MRResourceVisibility.values()) { + if (v.name().equals(visibility)) { + return true; + } + } + return false; + } + + public static MRResourceVisibility getVisibility(String visibilityStr) { + if (visibilityStr == null) { + return null; + } + visibilityStr = visibilityStr.toUpperCase(); + for (MRResourceVisibility v : MRResourceVisibility.values()) { + if (v.name().equals(visibilityStr)) { + return v; + } + } + return null; + } + + public static LocalResourceVisibility getYarnLocalResourceVisibility( + MRResourceVisibility visibility) { + switch (visibility) { + case PUBLIC: + return LocalResourceVisibility.PUBLIC; + case PRIVATE: + return LocalResourceVisibility.PRIVATE; + case APPLICATION: + return LocalResourceVisibility.APPLICATION; + default: + throw new IllegalArgumentException( + "Unsupport resourceVisility " + visibility.name()); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/SharedCacheConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/SharedCacheConfig.java index 34ec4e7..d79d7c9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/SharedCacheConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/SharedCacheConfig.java @@ -54,7 +54,6 @@ public void init(Configuration conf) { return; } - Collection configs = StringUtils.getTrimmedStringCollection( conf.get(MRJobConfig.SHARED_CACHE_MODE, MRJobConfig.SHARED_CACHE_MODE_DEFAULT)); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java index ada14db..02c7c5a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java @@ -31,6 +31,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.security.TokenCache; import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; /** * Manages internal configuration of the cache by the client for job submission. @@ -66,7 +67,7 @@ public static void determineTimestampsAndCacheVisibilities(Configuration job) */ public static void determineTimestampsAndCacheVisibilities(Configuration job, Map statCache) throws IOException { - determineTimestamps(job, statCache); + determineTimestampsAndSizes(job, statCache); determineCacheVisibilities(job, statCache); } @@ -80,8 +81,8 @@ public static void determineTimestampsAndCacheVisibilities(Configuration job, * @param job Configuration of a job. * @throws IOException */ - public static void determineTimestamps(Configuration job, - Map statCache) throws IOException { + public static void determineTimestampsAndSizes( + Configuration job, Map statCache) throws IOException { URI[] tarchives = DistributedCache.getCacheArchives(job); if (tarchives != null) { FileStatus status = getFileStatus(job, tarchives[0], statCache); @@ -161,26 +162,48 @@ public static void determineCacheVisibilities(Configuration job, Map statCache) throws IOException { URI[] tarchives = DistributedCache.getCacheArchives(job); if (tarchives != null) { - StringBuilder archiveVisibilities = - new StringBuilder(String.valueOf(isPublic(job, tarchives[0], statCache))); + StringBuilder archiveVisibilities = new StringBuilder( + determineResourceLocalVisibility( + job, tarchives[0], statCache).name()); for (int i = 1; i < tarchives.length; i++) { archiveVisibilities.append(","); - archiveVisibilities.append(String.valueOf(isPublic(job, tarchives[i], statCache))); + archiveVisibilities.append( + determineResourceLocalVisibility( + job, tarchives[i], statCache).name()); } setArchiveVisibilities(job, archiveVisibilities.toString()); } URI[] tfiles = DistributedCache.getCacheFiles(job); if (tfiles != null) { StringBuilder fileVisibilities = - new StringBuilder(String.valueOf(isPublic(job, tfiles[0], statCache))); + new StringBuilder( + determineResourceLocalVisibility( + job, tfiles[0], statCache).name()); for (int i = 1; i < tfiles.length; i++) { fileVisibilities.append(","); - fileVisibilities.append(String.valueOf(isPublic(job, tfiles[i], statCache))); + fileVisibilities.append( + determineResourceLocalVisibility( + job, tfiles[i], statCache).name()); } setFileVisibilities(job, fileVisibilities.toString()); } } - + + /** + * In most cases, we will have resources with visibilites to us share cache. + * However, there are some cases which uses distribute cache only without + * @param conf + * @param uri + * @param statCache + * @return + */ + private static LocalResourceVisibility determineResourceLocalVisibility( + Configuration conf, URI uri, Map statCache) + throws IOException { + return isPublic(conf, uri, statCache) ? + LocalResourceVisibility.PUBLIC : LocalResourceVisibility.APPLICATION; + } + /** * This is to check the public/private visibility of the archives to be * localized. @@ -197,11 +220,11 @@ static void setArchiveVisibilities(Configuration conf, String booleans) { * This is to check the public/private visibility of the files to be localized * * @param conf Configuration which stores the timestamp's - * @param booleans comma separated list of booleans (true - public) + * @param visibilities comma separated list of visibilities. Should * The order should be the same as the order in which the files are added. */ - static void setFileVisibilities(Configuration conf, String booleans) { - conf.set(MRJobConfig.CACHE_FILE_VISIBILITIES, booleans); + static void setFileVisibilities(Configuration conf, String visibilities) { + conf.set(MRJobConfig.CACHE_FILE_VISIBILITIES, visibilities); } /** @@ -230,7 +253,7 @@ static void setFileTimestamps(Configuration conf, String timestamps) { * Gets the file status for the given URI. If the URI is in the cache, * returns it. Otherwise, fetches it and adds it to the cache. */ - private static FileStatus getFileStatus(Configuration job, URI uri, + public static FileStatus getFileStatus(Configuration job, URI uri, Map statCache) throws IOException { FileSystem fileSystem = FileSystem.get(uri, job); return getFileStatus(fileSystem, uri, statCache); @@ -244,7 +267,7 @@ private static FileStatus getFileStatus(Configuration job, URI uri, * @return true if the path in the uri is visible to all, false otherwise * @throws IOException thrown if a file system operation fails */ - static boolean isPublic(Configuration conf, URI uri, + public static boolean isPublic(Configuration conf, URI uri, Map statCache) throws IOException { boolean isPublic = true; FileSystem fs = FileSystem.get(uri, conf); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java index 532e3ad..2f90527 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java @@ -29,6 +29,7 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import java.net.URI; @@ -219,7 +220,8 @@ public static void setCacheFiles(URI[] files, Configuration conf) { @Deprecated public static Path[] getLocalCacheFiles(Configuration conf) throws IOException { - return StringUtils.stringToPath(conf.getStrings(MRJobConfig.CACHE_LOCALFILES)); + return StringUtils.stringToPath( + conf.getStrings(MRJobConfig.CACHE_LOCALFILES)); } /** @@ -290,6 +292,32 @@ public static void addCacheArchive(URI uri, Configuration conf) { * will be created from the job's working directory to each file in the * parent directory. * + * @param uri + * @param visibility + * @param timeStamp + * @param fileLen + * @param conf + */ + public static void addCacheArchiveWithMeta( + URI uri, LocalResourceVisibility visibility, + long fileLen, long timeStamp, Configuration conf) { + appendToConfigItem(MRJobConfig.CACHE_ARCHIVES, uri.toString(), conf); + appendToConfigItem( + MRJobConfig.CACHE_ARCHIVES_VISIBILITIES, visibility.name(), conf); + appendToConfigItem( + MRJobConfig.CACHE_ARCHIVES_SIZES, Long.toString(fileLen), conf); + appendToConfigItem( + MRJobConfig.CACHE_ARCHIVES_TIMESTAMPS, Long.toString(timeStamp), conf); + } + + /** + * Add a file to be localized to the conf. The localized file will be + * downloaded to the execution node(s), and a link will created to the + * file from the job's working directory. If the last part of URI's path name + * is "*", then the entire parent directory will be localized and links + * will be created from the job's working directory to each file in the + * parent directory. + * * The access permissions of the file will determine whether the localized * file will be shared across jobs. If the file is not readable by other or * if any of its parent directories is not executable by other, then the @@ -313,6 +341,39 @@ public static void addCacheFile(URI uri, Configuration conf) { } /** + * Add a file to be localized to the conf. The localized file will be + * downloaded to the execution node(s), and a link will created to the + * file from the job's working directory. If the last part of URI's path name + * is "*", then the entire parent directory will be localized and links + * will be created from the job's working directory to each file in the + * parent directory. + * + * @param uri + * @param visibility + * @param timeStamp + * @param conf + */ + public static void addCacheFileWithMeta( + URI uri, LocalResourceVisibility visibility, + long timeStamp, long fileLen, Configuration conf) { + appendToConfigItem(MRJobConfig.CACHE_FILES, uri.toString(), conf); + appendToConfigItem( + MRJobConfig.CACHE_FILE_VISIBILITIES, visibility.name(), conf); + appendToConfigItem( + MRJobConfig.CACHE_FILE_TIMESTAMPS, Long.toString(timeStamp), conf); + appendToConfigItem( + MRJobConfig.CACHE_FILES_SIZES, Long.toString(fileLen), conf); + } + + private static void appendToConfigItem( + String key, String valueToAppend, Configuration conf) { + String existingVal = conf.get(key); + conf.set( + key, existingVal == null ? + valueToAppend : existingVal + "," + valueToAppend); + } + + /** * Add a file path to the current set of classpath entries. The file will * also be added to the cache. Intended to be used by user code. * @@ -401,7 +462,7 @@ public static void addFileToClassPath(Path file, Configuration conf, @Deprecated public static void addArchiveToClassPath(Path archive, Configuration conf) throws IOException { - addArchiveToClassPath(archive, conf, archive.getFileSystem(conf)); + addArchiveToClassPath(archive, conf, archive.getFileSystem(conf), true); } /** @@ -411,9 +472,10 @@ public static void addArchiveToClassPath(Path archive, Configuration conf) * @param archive Path of the archive to be added * @param conf Configuration that contains the classpath setting * @param fs FileSystem with respect to which {@code archive} should be interpreted. + * @param addToCache add archives to cache or classpath only */ - public static void addArchiveToClassPath - (Path archive, Configuration conf, FileSystem fs) + public static void addArchiveToClassPath( + Path archive, Configuration conf, FileSystem fs, boolean addToCache) throws IOException { String classpath = conf.get(MRJobConfig.CLASSPATH_ARCHIVES); conf.set(MRJobConfig.CLASSPATH_ARCHIVES, classpath == null ? archive @@ -468,6 +530,35 @@ public static boolean getSymlink(Configuration conf){ return true; } + /** + * Parse and return the LocalResourceVisibilities, return defaultVis when + * the visibilities are booleans. Old MR client will set visibilities + * as boolean. + * @param strs + * {@link LocalResourceVisibility} The defaultVis will be used as result + * when the string is boolean + * @return + */ + private static LocalResourceVisibility[] parseLocalResourceVisibilities( + String[] strs) { + if (null == strs) { + return null; + } + LocalResourceVisibility[] result = new LocalResourceVisibility[strs.length]; + for(int i=0; i < strs.length; ++i) { + if (strs[i].equalsIgnoreCase("true") + || strs[i].equalsIgnoreCase("false")) { + // The client before 2.9 will set visibility to be boolean. + // So keep the behavior unchanged while it's old client. + result[i] = strs[i].equalsIgnoreCase("true") ? + LocalResourceVisibility.PUBLIC : LocalResourceVisibility.PRIVATE; + } else { + result[i] = LocalResourceVisibility.valueOf(strs[i].toUpperCase()); + } + } + return result; + } + private static boolean[] parseBooleans(String[] strs) { if (null == strs) { return null; @@ -480,23 +571,31 @@ public static boolean getSymlink(Configuration conf){ } /** - * Get the booleans on whether the files are public or not. Used by - * internal DistributedCache and MapReduce code. + * Get the visibilities of cache files. If the value is Boolean, + * then it's set by old MR client. It will return + * MRJobConfig.FILES_VISIBILITY_DEFAULT in this case. + * It's Used by internal DistributedCache and MapReduce code. * @param conf The configuration which stored the timestamps * @return a string array of booleans */ - public static boolean[] getFileVisibilities(Configuration conf) { - return parseBooleans(conf.getStrings(MRJobConfig.CACHE_FILE_VISIBILITIES)); + public static LocalResourceVisibility[] getFileVisibilities( + Configuration conf) { + return parseLocalResourceVisibilities( + conf.getStrings(MRJobConfig.CACHE_FILE_VISIBILITIES)); } /** - * Get the booleans on whether the archives are public or not. Used by - * internal DistributedCache and MapReduce code. + * Get the visibilities of cache files. If the value is Boolean, which means + * it's set by old MR client. The method will return + * MRJobConfig.ARCHIEVES_VISIBILITY_DEFAULT in this case. + * Used by internal DistributedCache and MapReduce code. * @param conf The configuration which stored the timestamps * @return a string array of booleans */ - public static boolean[] getArchiveVisibilities(Configuration conf) { - return parseBooleans(conf.getStrings(MRJobConfig.CACHE_ARCHIVES_VISIBILITIES)); + public static LocalResourceVisibility[] getArchiveVisibilities( + Configuration conf) { + return parseLocalResourceVisibilities( + conf.getStrings(MRJobConfig.CACHE_ARCHIVES_VISIBILITIES)); } /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/MRResourceUtil.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/MRResourceUtil.java new file mode 100644 index 0000000..58bf2f1 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/MRResourceUtil.java @@ -0,0 +1,271 @@ +/** + * 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.mapreduce.util; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.MRResource; +import org.apache.hadoop.mapreduce.MRResourceType; +import org.apache.hadoop.mapreduce.MRResourceVisibility; +import org.apache.hadoop.util.GenericOptionsParser; +import org.apache.hadoop.util.ResourceWithVisibilitySetting; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; + +import static org.apache.hadoop.mapreduce.MRResourceType.ARCHIVE; +import static org.apache.hadoop.mapreduce.MRResourceType.FILE; +import static org.apache.hadoop.mapreduce.MRResourceType.JOBJAR; +import static org.apache.hadoop.mapreduce.MRResourceType.LIBJAR; + +/** + * MapReduce utility class. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class MRResourceUtil { + + private MRResourceUtil() {} + + /** + * Generate resource from config string. Config format should like + * "$path" or "$path:${visibility} + * Visibility should be one fo following value + * {@link MRResourceVisibility#PUBLIC}, + * {@link MRResourceVisibility#APPLICATION}, + * {@link MRResourceVisibility#PRIVATE}" + * @param confString + * @param resourceType + * @param defaultVisibility + * @return + */ + public static MRResource getResourceFromOptionStr( + String confString, MRResourceType resourceType, + MRResourceVisibility defaultVisibility) { + ResourceWithVisibilitySetting resource = + ResourceWithVisibilitySetting.deserialize(confString); + MRResourceVisibility visibility = + MRResourceVisibility.getVisibility( + resource.getVisibilitySettings()); + visibility = visibility == null ? defaultVisibility : visibility; + return new MRResource(resourceType, resource.getPathStr(), visibility); + } + + /** + * Get resource specified by -libjars, -files, -archives and jobjar. + * @param resourceType + * @param configuration + * @return + */ + public static List getCmdConfiguredResourceFromMRConfig( + MRResourceType resourceType, Configuration configuration) { + MRResourceVisibility defaultVisibility = + getVisibilityFromMRConfig(resourceType, configuration); + switch (resourceType) { + case JOBJAR: + return createResourceCollectionsInternal( + JOBJAR, defaultVisibility, JobContext.JAR, configuration); + case ARCHIVE: + return createResourceCollectionsInternal( + ARCHIVE, defaultVisibility, + GenericOptionsParser.TMP_ARCHIVES_CONF_KEY, configuration); + case LIBJAR: + return createResourceCollectionsInternal( + LIBJAR, defaultVisibility, + GenericOptionsParser.TMP_LIBJARS_CONF_KEY, configuration); + case FILE: + return createResourceCollectionsInternal( + FILE, defaultVisibility, + GenericOptionsParser.TMP_FILES_CONF_KEY, configuration); + default: + throw new IllegalArgumentException( + "Unkown resource type:" + resourceType); + } + } + + /** + * Get resources specified from cmd line and resources that have been + * programmatically specified for the shared cache via the Job API. + * @return + */ + public static List getResourceFromMRConfig( + MRResourceType resourceType, Configuration configuration) { + List resourceList = + getCmdConfiguredResourceFromMRConfig(resourceType, configuration); + if (resourceType != JOBJAR) { + // We might have resources not passed in by CMD configuration if it's not + // job jar. + resourceList.addAll( + getSharedCacheResourceFromMRConfig( + resourceType, configuration)); + } + return resourceList; + } + + /** + * Get resources that have been programmatically specified for the shared + * cache via the Job API. + * @param resourceType + * @param configuration + * @return + */ + public static List getSharedCacheResourceFromMRConfig( + MRResourceType resourceType, Configuration configuration) { + MRResourceVisibility defaultVisibility = + getVisibilityFromMRConfig(resourceType, configuration); + switch (resourceType) { + case ARCHIVE: + return createResourceCollectionsInternal( + ARCHIVE, defaultVisibility, + MRJobConfig.ARCHIVES_FOR_SHARED_CACHE, configuration); + case LIBJAR: + return createResourceCollectionsInternal( + LIBJAR, defaultVisibility, + MRJobConfig.FILES_FOR_CLASSPATH_AND_SHARED_CACHE, configuration); + case FILE: + return createResourceCollectionsInternal( + FILE, defaultVisibility, + MRJobConfig.FILES_FOR_SHARED_CACHE, configuration); + case JOBJAR: + // There are no shared cached resource added to configuration + default: + throw new IllegalArgumentException( + "Unkown resource type:" + resourceType); + } + } + + public static MRResource getJobJar(Configuration configuration) { + Collection jobJarResource = + getCmdConfiguredResourceFromMRConfig( + MRResourceType.JOBJAR, configuration); + if (jobJarResource == null || jobJarResource.isEmpty()) { + return null; + } + return jobJarResource.iterator().next(); + } + + private static List createResourceCollectionsInternal( + MRResourceType resourceType, MRResourceVisibility visibility, + String confKey, Configuration configuration) { + List resourceList = new ArrayList(); + Collection resourceStrCollection = + configuration.getStringCollection(confKey); + for (String resourceStr : resourceStrCollection) { + MRResource resource = getResourceFromOptionStr( + resourceStr, resourceType, visibility); + resourceList.add(resource); + } + return resourceList; + } + + public static MRResourceVisibility getVisibilityFromMRConfig( + MRResourceType resourceType, Configuration conf) { + MRResourceVisibility visibility = MRResourceVisibility.APPLICATION; + switch (resourceType) { + case JOBJAR: + visibility = getResourceVisibility( + conf, MRJobConfig.JOBJAR_VISIBILITY, + MRJobConfig.JOBJAR_VISIBILITY_DEFAULT); + break; + case LIBJAR: + visibility = getResourceVisibility( + conf, MRJobConfig.LIBJARS_VISIBILITY, + MRJobConfig.LIBJARS_VISIBILITY_DEFAULT); + break; + case FILE: + visibility = getResourceVisibility( + conf, MRJobConfig.FILES_VISIBILITY, + MRJobConfig.FILES_VISIBILITY_DEFAULT); + break; + case ARCHIVE: + visibility = getResourceVisibility( + conf, MRJobConfig.ARCHIEVS_VISIBILITY, + MRJobConfig.ARCHIEVES_VISIBILITY_DEFAULT); + break; + default: + break; + } + return visibility; + } + + public static MRResourceVisibility getResourceVisibility( + Configuration conf, String key, MRResourceVisibility defaultVal) { + String visibilityStr = conf.get(key); + if (visibilityStr == null) { + return defaultVal; + } + return MRResourceVisibility.getVisibility(visibilityStr); + } + + /** + * This method should be called after job jar is uploaded to HDFS. + * @param conf + * @return + */ + public static LocalResourceVisibility getJobJarYarnLocalVisibility( + Configuration conf) { + String visibilityStr = conf.get( + MRJobConfig.CACHE_JOBJAR_VISIBILITY); + if (visibilityStr == null) { + // The visibility string will be null before 2.9 as it uses + // {@link MRJobConf.JOBJAR_VISIBILITY} to determined jar visibility. + // We make MRJobConf.JOBJAR_VISIBILITY user facing only and + // use CACHE_JOBJAR_VISIBILITY to determine job jar visibility after 2.9. + String oldFormatVis = conf.get(MRJobConfig.JOBJAR_VISIBILITY); + if (oldFormatVis != null) { + return oldFormatVis.equalsIgnoreCase("true") ? + LocalResourceVisibility.PUBLIC : + LocalResourceVisibility.APPLICATION; + } + return MRResourceVisibility.getYarnLocalResourceVisibility( + MRJobConfig.JOBJAR_VISIBILITY_DEFAULT); + } + return MRResourceVisibility.getYarnLocalResourceVisibility( + MRResourceVisibility.getVisibility(visibilityStr)); + } + + public static LocalResourceVisibility getFilesYarnLocalVisibility( + Configuration conf) { + return MRResourceVisibility.getYarnLocalResourceVisibility( + getResourceVisibility( + conf, MRJobConfig.FILES_VISIBILITY, + MRJobConfig.FILES_VISIBILITY_DEFAULT)); + } + + public static LocalResourceVisibility getArchivesYarnLocalVisibility( + Configuration conf) { + return MRResourceVisibility.getYarnLocalResourceVisibility( + getResourceVisibility( + conf, MRJobConfig.ARCHIEVS_VISIBILITY, + MRJobConfig.ARCHIEVES_VISIBILITY_DEFAULT)); + } + + public static LocalResourceVisibility getLibjarsYarnLocalVisibility( + Configuration conf) { + return MRResourceVisibility.getYarnLocalResourceVisibility( + getResourceVisibility( + conf, MRJobConfig.LIBJARS_VISIBILITY, + MRJobConfig.LIBJARS_VISIBILITY_DEFAULT)); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml index 1ba82d2..f9eb144 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml @@ -2191,4 +2191,102 @@ + + mapreduce.job.jobjar.visibility + application + + This parameter controls the visibility of the localized job jar on the node + manager. It should be one of + MRResourceVisibility{PUBLIC, PRIVATE, APPLICATION}. This is an user facing + config item. + + + + + mapreduce.job.libjars.visibility + application + + This parameter controls the visibility of the localized lib jars on + the node manager. It should be one of + #MRResourceVisibility{PUBLIC, PRIVATE, APPLICATION}. + Note that it will only work for resources whose visibility is not set + implicitly. E.g. the following resource will be `public`: + -libjars a.jar::public + + + + + mapreduce.job.files.visibility + application + + This parameter controls the visibility of the localized files on + the node manager. It should be one of + #MRResourceVisibility{PUBLIC, PRIVATE, APPLICATION}. + Note that it will only work for resources whose visibility is not set + implicitly. E.g. the following resource will be `public`: + -files a.file::public . + + + + + mapreduce.job.archives.visibility + application + + This parameter controls the visibility of the localized archives on + the node manager. It should be one of + #MRResourceVisibility{PUBLIC, PRIVATE, APPLICATION}. + Note that it will only work for resources whose visibility is not set + implicitly. E.g. the following resource will be `public`: + -archives a.zip::public . + + + + + mapreduce.job.log4j-properties-file.visibility + application + + This parameter controls the visibility of the log4j-properties-file on + the node manager. It should be one of + #MRResourceVisibility{PUBLIC, PRIVATE, APPLICATION}. + + + + + mapreduce.job.user.staging.dirname + .staging + + This flag is used to set the last dir name of users's staging path. + A MR job's staging dir is defined in + {\@link org.apache.hadoop.mapreduce.v2.util.MRApps#getStagingAreaDir}. + ${MRJobConfig.MR_AM_STAGING_DIR}/${user}/${MR_USER_STAGING_DIR_NAME}; + E.g. user1 launched two jobs: job-1234 and job-abcd. Their staging dir + will be + ${MRJobConfig.MR_AM_STAGING_DIR}/user1/${MR_USER_STAGING_DIR_NAME}/job-1234 + ${MRJobConfig.MR_AM_STAGING_DIR}/user1/${MR_USER_STAGING_DIR_NAME}/job-abcd + + + + + mapreduce.job.user.staging.dirname + .staging + + This flag is used to set the last dir name of users's staging path. + A MR job's staging dir is defined in + {\@link org.apache.hadoop.mapreduce.v2.util.MRApps#getStagingAreaDir}. + ${MRJobConfig.MR_AM_STAGING_DIR}/${user}/${MR_USER_STAGING_DIR_NAME}; + E.g. user1 launched two jobs: job-1234 and job-abcd. Their staging dir + will be + ${MRJobConfig.MR_AM_STAGING_DIR}/user1/${MR_USER_STAGING_DIR_NAME}/job-1234 + ${MRJobConfig.MR_AM_STAGING_DIR}/user1/${MR_USER_STAGING_DIR_NAME}/job-abcd + + + + + mapreduce.job.staging.dir.permission + 0700 + + Permission of MR job staging dir, default is 0700. + + + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java index 8033897..f2137b6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java @@ -18,18 +18,20 @@ package org.apache.hadoop.mapreduce; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyMap; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.times; -import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import java.io.IOException; import java.net.URI; import java.util.Arrays; -import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -44,6 +46,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies; import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.util.MRResourceUtil; import org.junit.Assert; import org.junit.Test; import org.mockito.verification.VerificationMode; @@ -61,7 +64,8 @@ public void testStringToPath() throws IOException { Assert.assertEquals("Failed: absolute, no scheme, with fragment", "/testWithFragment.txt", - uploader.stringToPath("/testWithFragment.txt#fragment.txt").toString()); + uploader.stringToPath( + "/testWithFragment.txt#fragment.txt").toString()); Assert.assertEquals("Failed: absolute, with scheme, with fragment", "file:/testWithFragment.txt", @@ -70,7 +74,8 @@ public void testStringToPath() throws IOException { Assert.assertEquals("Failed: relative, no scheme, with fragment", "testWithFragment.txt", - uploader.stringToPath("testWithFragment.txt#fragment.txt").toString()); + uploader.stringToPath( + "testWithFragment.txt#fragment.txt").toString()); Assert.assertEquals("Failed: relative, no scheme, no fragment", "testWithFragment.txt", @@ -209,7 +214,10 @@ public void testOverSingleResourceMBLimit() throws IOException { destinationPathPrefix + "tmpFiles2.txt", destinationPathPrefix + "tmpFiles3.txt", destinationPathPrefix + "tmpFiles4.txt", - "file:///libjars-submit-dir/libjars/*" }; + destinationPathPrefix + "tmpjars0.jar", + destinationPathPrefix + "tmpjars1.jar", + destinationPathPrefix + "tmpjars2.jar", + destinationPathPrefix + "tmpjars3.jar"}; private String[] expectedArchivesNoFrags = { destinationPathPrefix + "tmpArchives0.tgz", @@ -219,8 +227,8 @@ public void testOverSingleResourceMBLimit() throws IOException { { destinationPathPrefix + "tmpArchives0.tgz#tmpArchivesfragment0.tgz", destinationPathPrefix + "tmpArchives1.tgz#tmpArchivesfragment1.tgz" }; - private String jobjarSubmitDir = "/jobjar-submit-dir"; - private String basicExpectedJobJar = jobjarSubmitDir + "/job.jar"; + private String jobjarSubmitDir = "hdfs:///jobjar-submit-dir"; + private String basicExpectedJobJar = destinationPathPrefix + "jobjar.jar"; @Test public void testPathsWithNoFragNoSchemeRelative() throws IOException { @@ -410,20 +418,24 @@ private void runTmpResourcePathTest(JobResourceUploader uploader, private void uploadResources(JobResourceUploader uploader, Job job) throws IOException { Configuration conf = job.getConfiguration(); - Collection files = conf.getStringCollection("tmpfiles"); - Collection libjars = conf.getStringCollection("tmpjars"); - Collection archives = conf.getStringCollection("tmparchives"); Map statCache = new HashMap<>(); Map fileSCUploadPolicies = new HashMap<>(); - String jobJar = job.getJar(); - uploader.uploadFiles(job, files, new Path("/files-submit-dir"), null, - (short) 3, fileSCUploadPolicies, statCache); - uploader.uploadArchives(job, archives, new Path("/archives-submit-dir"), - null, (short) 3, fileSCUploadPolicies, statCache); - uploader.uploadLibJars(job, libjars, new Path("/libjars-submit-dir"), null, - (short) 3, fileSCUploadPolicies, statCache); - uploader.uploadJobJar(job, jobJar, new Path(jobjarSubmitDir), (short) 3, - statCache); + + List filesList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.FILE, conf); + List libjarsList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.LIBJAR, conf); + List archivesList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.ARCHIVE, conf); + MRResource jobJar = MRResourceUtil.getJobJar(conf); + + uploader.uploadFiles(job, new Path("/files-submit-dir"), filesList, + (short) 3); + uploader.uploadArchives(job, new Path("/archives-submit-dir"), + archivesList, (short) 3); + uploader.uploadLibJars( + job, new Path("/libjars-submit-dir"), libjarsList, (short) 3); + uploader.uploadJobJar(job, new Path(jobjarSubmitDir), jobJar, (short) 3); } private void validateResourcePaths(Job job, String[] expectedFiles, @@ -474,11 +486,19 @@ private void runLimitsTest(ResourceConf rlConf, boolean checkShouldSucceed, when(mockedStatus.isDirectory()).thenReturn(false); Map statCache = new HashMap(); try { + + List filesList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.FILE, conf); + List libjarsList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.LIBJAR, conf); + List archivesList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.ARCHIVE, conf); + MRResource jobJarResource = MRResourceUtil.getJobJar(conf); uploader.checkLocalizationLimits(conf, - conf.getStringCollection("tmpfiles"), - conf.getStringCollection("tmpjars"), - conf.getStringCollection("tmparchives"), - conf.getJar(), statCache); + filesList, + libjarsList, + archivesList, + jobJarResource); Assert.assertTrue("Limits check succeeded when it should have failed.", checkShouldSucceed); } catch (IOException e) { @@ -741,12 +761,29 @@ private String buildPathStringSub(String pathPrefix, String processedPath, } @Override - FileStatus getFileStatus(Map statCache, Configuration job, + FileStatus getFileStatus(Configuration job, Path p) throws IOException { return mockedStatus; } @Override + MRResourceInfo checkAndUploadMRResource( + Job job, Path submitJobDir, + MRResource resource, short submitReplication) + throws IOException { + MRResourceInfo resourceInfo = super.checkAndUploadMRResource( + job, submitJobDir, resource, submitReplication); + MRResourceInfo spyInfo = spy(resourceInfo); + doReturn(mockedStatus.getLen()).when(spyInfo).getUriToUseFileSize( + any(Configuration.class), + anyMap()); + doReturn(mockedStatus.getModificationTime()) + .when(spyInfo).getUriToUseTimestamp( + any(Configuration.class), anyMap()); + return spyInfo; + } + + @Override boolean mkdirs(FileSystem fs, Path dir, FsPermission permission) throws IOException { // Do nothing. Stubbed out to avoid side effects. We don't actually need @@ -759,12 +796,5 @@ Path copyRemoteFiles(Path parentDir, Path originalPath, Configuration conf, short replication) throws IOException { return new Path(destinationPathPrefix + originalPath.getName()); } - - @Override - void copyJar(Path originalJarPath, Path submitJarFile, short replication) - throws IOException { - // Do nothing. Stubbed out to avoid side effects. We don't actually need - // to copy the jar to the remote fs. - } } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploaderWithSharedCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploaderWithSharedCache.java index 5555043..660611e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploaderWithSharedCache.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploaderWithSharedCache.java @@ -85,8 +85,9 @@ public static void setup() throws IOException { localFs = FileSystem.getLocal(conf); testRootDir = new Path("target", - TestJobResourceUploaderWithSharedCache.class.getName() + "-tmpDir") - .makeQualified(localFs.getUri(), localFs.getWorkingDirectory()); + TestJobResourceUploaderWithSharedCache.class.getName() + + "-tmpDir").makeQualified( + localFs.getUri(), localFs.getWorkingDirectory()); dfs = new MiniDFSCluster.Builder(conf).numDataNodes(1).format(true).build(); remoteFs = dfs.getFileSystem(); } @@ -168,6 +169,14 @@ public void testSharedCacheDisabled() throws Exception { public void testSharedCacheEnabled() throws Exception { JobConf jobConf = createJobConf(); jobConf.set(MRJobConfig.SHARED_CACHE_MODE, "enabled"); + jobConf.set( + MRJobConfig.LIBJARS_VISIBILITY, MRResourceVisibility.PUBLIC.name()); + jobConf.set( + MRJobConfig.ARCHIEVS_VISIBILITY, MRResourceVisibility.PUBLIC.name()); + jobConf.set( + MRJobConfig.FILES_VISIBILITY, MRResourceVisibility.PUBLIC.name()); + jobConf.set( + MRJobConfig.JOBJAR_VISIBILITY, MRResourceVisibility.PUBLIC.name()); Job job = new Job(jobConf); job.setJobID(new JobID("567789", 1)); @@ -182,6 +191,15 @@ public void testSharedCacheEnabledWithJobJarInSharedCache() throws Exception { JobConf jobConf = createJobConf(); jobConf.set(MRJobConfig.SHARED_CACHE_MODE, "enabled"); + + jobConf.set( + MRJobConfig.LIBJARS_VISIBILITY, MRResourceVisibility.PUBLIC.name()); + jobConf.set( + MRJobConfig.ARCHIEVS_VISIBILITY, MRResourceVisibility.PUBLIC.name()); + jobConf.set( + MRJobConfig.FILES_VISIBILITY, MRResourceVisibility.PUBLIC.name()); + jobConf.set( + MRJobConfig.JOBJAR_VISIBILITY, MRResourceVisibility.PUBLIC.name()); Job job = new Job(jobConf); job.setJobID(new JobID("567789", 1)); @@ -195,6 +213,10 @@ public void testSharedCacheEnabledWithJobJarInSharedCache() public void testSharedCacheArchivesAndLibjarsEnabled() throws Exception { JobConf jobConf = createJobConf(); jobConf.set(MRJobConfig.SHARED_CACHE_MODE, "archives,libjars"); + jobConf.set( + MRJobConfig.LIBJARS_VISIBILITY, MRResourceVisibility.PUBLIC.name()); + jobConf.set( + MRJobConfig.ARCHIEVS_VISIBILITY, MRResourceVisibility.PUBLIC.name()); Job job = new Job(jobConf); job.setJobID(new JobID("567789", 1)); @@ -250,7 +272,8 @@ private void uploadFilesToRemoteFS(Job job, JobConf jobConf, jobConf.set("tmpfiles", secondFile.toString()); // Create jars with a single file inside them. - Path firstJar = makeJar(new Path(testRootDir, "distributed.first.jar"), 1); + Path firstJar = makeJar( + new Path(testRootDir, "distributed.first.jar"), 1); Path secondJar = makeJar(new Path(testRootDir, "distributed.second.jar"), 2); @@ -273,8 +296,10 @@ private void uploadFilesToRemoteFS(Job job, JobConf jobConf, jobConf.set("tmpjars", secondJar.toString() + "," + thirdJar.toString()); - Path firstArchive = makeArchive("first-archive.zip", "first-file"); - Path secondArchive = makeArchive("second-archive.zip", "second-file"); + Path firstArchive = makeArchive( + "first-archive.zip", "first-file"); + Path secondArchive = makeArchive( + "second-archive.zip", "second-file"); // Add archives to job conf via distributed cache API as well as command // line @@ -331,7 +356,7 @@ private Path createTempFile(String filename, String contents) FSDataOutputStream os = localFs.create(path); os.writeBytes(contents); os.close(); - localFs.setPermission(path, new FsPermission("700")); + localFs.setPermission(path, new FsPermission("705")); return path; } @@ -345,7 +370,7 @@ private Path makeJar(Path p, int index) throws FileNotFoundException, jos.write(("inside the jar!" + index).getBytes()); jos.closeEntry(); jos.close(); - localFs.setPermission(p, new FsPermission("700")); + localFs.setPermission(p, new FsPermission("705")); return p; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobSubmissionFiles.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobSubmissionFiles.java index ab3f7a0..20e660c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobSubmissionFiles.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobSubmissionFiles.java @@ -30,7 +30,10 @@ import java.io.IOException; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; /** @@ -139,4 +142,79 @@ public void testGetStagingDirWhenShortFileOwnerNameAndShortUserName() assertEquals(stagingPath, JobSubmissionFiles.getStagingDir(cluster, conf, user)); } + + @Test + public void testSetStagingDirDefaultPermission() + throws Exception { + Cluster cluster = mock(Cluster.class); + Configuration conf = new Configuration(); + Path stagingPath = mock(Path.class); + UserGroupInformation user = UserGroupInformation + .createUserForTesting(USER_1, GROUP_NAMES); + assertEquals(USER_1, user.getUserName()); + FileSystem fs = new FileSystemTestHelper.MockFileSystem(); + when(cluster.getStagingAreaDir()).thenReturn(stagingPath); + when(stagingPath.getFileSystem(conf)).thenReturn(fs); + + //Staging directory owner full principal name is in lower case. + String stagingDirOwner = USER_1.toLowerCase(); + FileStatus fileStatus = new FileStatus(1, true, 1, 1, 100L, 100L, + FsPermission.getDefault(), stagingDirOwner, stagingDirOwner, + stagingPath); + when(fs.getFileStatus(stagingPath)).thenReturn(fileStatus); + assertEquals(stagingPath, + JobSubmissionFiles.getStagingDir(cluster, conf, user)); + + FsPermission defaultPerm = + FsPermission.createImmutable( + MRJobConfig.DEFAULT_MR_JOB_STAGING_DIR_PERMISSION); + verify(((FileSystemTestHelper.MockFileSystem) fs).getRawFileSystem(), + times(1)).setPermission(stagingPath, defaultPerm); + } + + @Test + public void testSetStagingDirCustomPermission() + throws Exception { + Cluster cluster = mock(Cluster.class); + Configuration conf = new Configuration(); + Path stagingPath = mock(Path.class); + UserGroupInformation user = UserGroupInformation + .createUserForTesting(USER_1, GROUP_NAMES); + assertEquals(USER_1, user.getUserName()); + FileSystem fs = new FileSystemTestHelper.MockFileSystem(); + when(cluster.getStagingAreaDir()).thenReturn(stagingPath); + when(stagingPath.getFileSystem(conf)).thenReturn(fs); + conf.set(MRJobConfig.MR_JOB_STAGING_DIR_PERMISSION, "0711"); + + //Staging directory owner full principal name is in lower case. + String stagingDirOwner = USER_1.toLowerCase(); + FileStatus fileStatus = new FileStatus(1, true, 1, 1, 100L, 100L, + FsPermission.getDefault(), stagingDirOwner, stagingDirOwner, + stagingPath); + when(fs.getFileStatus(stagingPath)).thenReturn(fileStatus); + assertEquals(stagingPath, + JobSubmissionFiles.getStagingDir(cluster, conf, user)); + + FsPermission permission = + FsPermission.createImmutable((short) 0711); + verify(((FileSystemTestHelper.MockFileSystem) fs).getRawFileSystem(), + times(1)).setPermission(stagingPath, permission); + } + + @Test + public void testGetJobDistCacheLibjarsPath() { + Path submitJobDir = new Path("/JobDir"); + assertTrue( + JobSubmissionFiles.getJobDistCacheLibjarsPath( + submitJobDir, MRResourceVisibility.PUBLIC). + toString().equals("/JobDir/public/libjars")); + assertTrue( + JobSubmissionFiles.getJobDistCacheLibjarsPath( + submitJobDir, MRResourceVisibility.APPLICATION). + toString().equals("/JobDir/application/libjars")); + assertTrue( + JobSubmissionFiles.getJobDistCacheLibjarsPath( + submitJobDir, MRResourceVisibility.PRIVATE). + toString().equals("/JobDir/private/libjars")); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestMRResource.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestMRResource.java new file mode 100644 index 0000000..27233f9 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestMRResource.java @@ -0,0 +1,78 @@ +/** + * 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.mapreduce; + +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.Test; + +/** + * Test class for MRResource. + */ +public class TestMRResource { + + @Test + public void testGetResourceSubmissionParentDir() { + Path jobDir = new Path("/jobdir"); + MRResource jobJar = new MRResource( + MRResourceType.JOBJAR, "/testjojar", MRResourceVisibility.PUBLIC); + assertTrue(jobJar. + getResourceSubmissionParentDir(jobDir) + .toString() + .equals("/jobdir/public")); + MRResource file = new MRResource( + MRResourceType.FILE, "/file1", MRResourceVisibility.APPLICATION); + assertTrue(file.getResourceSubmissionParentDir(jobDir). + toString().equals("/jobdir/application/files")); + MRResource archive = new MRResource( + MRResourceType.ARCHIVE, "/archive1", MRResourceVisibility.PRIVATE); + assertTrue(archive.getResourceSubmissionParentDir(jobDir). + toString().equals("/jobdir/private/archives")); + MRResource libjar = new MRResource( + MRResourceType.LIBJAR, "/jar0", MRResourceVisibility.APPLICATION); + assertTrue(libjar.getResourceSubmissionParentDir(jobDir). + toString().equals("/jobdir/application/libjars")); + } + + @Test + public void testGetLocalResourceUri() throws Exception { + Configuration conf = new Configuration(); + // Local resource + String localPathStr = "/testjojar"; + MRResource jobJar = new MRResource( + MRResourceType.JOBJAR, localPathStr, MRResourceVisibility.PUBLIC); + Path expectPath = FileSystem.getLocal(conf).makeQualified( + new Path(localPathStr)); + assertTrue(jobJar.getResourceUri(conf).toString().equals( + expectPath.toString())); + } + + @Test + public void testGetNonLocalResourceUri() throws Exception { + Configuration conf = new Configuration(); + String pathStr = "viewfs://nn1/testjojar/"; + MRResource jobJar = new MRResource( + MRResourceType.JOBJAR, pathStr, MRResourceVisibility.APPLICATION); + assertTrue(jobJar.getResourceUri(conf).toString().equals( + "viewfs://nn1/testjojar")); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestMRResourceVisibility.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestMRResourceVisibility.java new file mode 100644 index 0000000..e460358 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestMRResourceVisibility.java @@ -0,0 +1,60 @@ +/** + * 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.mapreduce; + +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; +import org.junit.Test; + +/** + * Test class for MRResourceVisibility. + */ +public class TestMRResourceVisibility { + + @Test + public void testGetVisibility() { + // lower case + MRResourceVisibility visibility = + MRResourceVisibility.getVisibility("public"); + assertTrue(visibility.name().equals(MRResourceVisibility.PUBLIC.name())); + + // Upper case + visibility = + MRResourceVisibility.getVisibility("PUBLIC"); + assertTrue(visibility.name().equals(MRResourceVisibility.PUBLIC.name())); + } + + @Test + public void testGetYarnLocalResourceVisibility() { + testGetYarnLocalResourceInternal( + MRResourceVisibility.PUBLIC, LocalResourceVisibility.PUBLIC); + testGetYarnLocalResourceInternal( + MRResourceVisibility.APPLICATION, LocalResourceVisibility.APPLICATION); + testGetYarnLocalResourceInternal( + MRResourceVisibility.PRIVATE, LocalResourceVisibility.PRIVATE); + } + + private void testGetYarnLocalResourceInternal( + MRResourceVisibility input, LocalResourceVisibility expectOutput) { + LocalResourceVisibility vis = + MRResourceVisibility.getYarnLocalResourceVisibility(input); + assertTrue(vis.name().equals(expectOutput.name())); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java index a61e938..64ea3d5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java @@ -34,7 +34,6 @@ import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.MRJobConfig; - import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -95,7 +94,8 @@ public void testDetermineTimestamps() throws IOException { Configuration jobConf = job.getConfiguration(); Map statCache = new HashMap<>(); - ClientDistributedCacheManager.determineTimestamps(jobConf, statCache); + ClientDistributedCacheManager.determineTimestampsAndSizes( + jobConf, statCache); FileStatus firstStatus = statCache.get(firstCacheFile.toUri()); FileStatus secondStatus = statCache.get(secondCacheFile.toUri()); @@ -114,7 +114,8 @@ public void testDetermineTimestamps() throws IOException { job.addCacheFile(new Path(TEST_VISIBILITY_CHILD_DIR, "*").toUri()); jobConf = job.getConfiguration(); statCache.clear(); - ClientDistributedCacheManager.determineTimestamps(jobConf, statCache); + ClientDistributedCacheManager.determineTimestampsAndSizes( + jobConf, statCache); FileStatus thirdStatus = statCache.get(TEST_VISIBILITY_CHILD_DIR.toUri()); @@ -155,7 +156,8 @@ public void testDetermineCacheVisibilities() throws IOException { // between wrong and missing assertEquals("The file paths were not found to be publicly visible " + "even though the full path is publicly accessible", - "true,true", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES)); + "PUBLIC,PUBLIC", jobConf.get( + MRJobConfig.CACHE_FILE_VISIBILITIES)); checkCacheEntries(statCache, null, firstCacheFile, relativePath); job = Job.getInstance(conf); @@ -169,7 +171,8 @@ public void testDetermineCacheVisibilities() throws IOException { // between wrong and missing assertEquals("The file path was not found to be publicly visible " + "even though the full path is publicly accessible", - "true", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES)); + "PUBLIC", + jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES)); checkCacheEntries(statCache, null, wildcardPath.getParent()); Path qualifiedParent = fs.makeQualified(TEST_VISIBILITY_PARENT_DIR); @@ -187,7 +190,8 @@ public void testDetermineCacheVisibilities() throws IOException { // between wrong and missing assertEquals("The file paths were found to be publicly visible " + "even though the parent directory is not publicly accessible", - "false,false", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES)); + "APPLICATION,APPLICATION", + jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES)); checkCacheEntries(statCache, qualifiedParent, firstCacheFile, relativePath); @@ -202,7 +206,8 @@ public void testDetermineCacheVisibilities() throws IOException { // between wrong and missing assertEquals("The file path was found to be publicly visible " + "even though the parent directory is not publicly accessible", - "false", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES)); + "APPLICATION", + jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES)); checkCacheEntries(statCache, qualifiedParent, wildcardPath.getParent()); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestDistributedCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestDistributedCache.java index 14f4020..3268852 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestDistributedCache.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestDistributedCache.java @@ -17,10 +17,13 @@ */ package org.apache.hadoop.mapreduce.filecache; +import java.net.URI; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.junit.Test; import static org.junit.Assert.*; @@ -45,7 +48,8 @@ public void testAddFileToClassPath() throws Exception { DistributedCache.addFileToClassPath(new Path("file:///a"), conf); assertEquals("The mapreduce.job.classpath.files property was not " - + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES)); + + "set correctly", "file:/a", + conf.get(MRJobConfig.CLASSPATH_FILES)); assertEquals("The mapreduce.job.cache.files property was not set " + "correctly", "file:///a", conf.get(MRJobConfig.CACHE_FILES)); @@ -68,13 +72,16 @@ public void testAddFileToClassPath() throws Exception { // Expected } - DistributedCache.addFileToClassPath(new Path("file:///a"), conf, fs); + DistributedCache.addFileToClassPath( + new Path("file:///a"), conf, fs); assertEquals("The mapreduce.job.classpath.files property was not " - + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES)); + + "set correctly", "file:/a", + conf.get(MRJobConfig.CLASSPATH_FILES)); assertEquals("The mapreduce.job.cache.files property was not set " + "correctly", "file:///a", conf.get(MRJobConfig.CACHE_FILES)); - DistributedCache.addFileToClassPath(new Path("file:///b"), conf, fs); + DistributedCache.addFileToClassPath( + new Path("file:///b"), conf, fs); assertEquals("The mapreduce.job.classpath.files property was not " + "set correctly", "file:/a,file:/b", conf.get(MRJobConfig.CLASSPATH_FILES)); @@ -92,13 +99,16 @@ public void testAddFileToClassPath() throws Exception { // Expected } - DistributedCache.addFileToClassPath(new Path("file:///a"), conf, fs, true); + DistributedCache.addFileToClassPath( + new Path("file:///a"), conf, fs, true); assertEquals("The mapreduce.job.classpath.files property was not " - + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES)); + + "set correctly", "file:/a", + conf.get(MRJobConfig.CLASSPATH_FILES)); assertEquals("The mapreduce.job.cache.files property was not set " + "correctly", "file:///a", conf.get(MRJobConfig.CACHE_FILES)); - DistributedCache.addFileToClassPath(new Path("file:///b"), conf, fs, true); + DistributedCache.addFileToClassPath( + new Path("file:///b"), conf, fs, true); assertEquals("The mapreduce.job.classpath.files property was not " + "set correctly", "file:/a,file:/b", conf.get(MRJobConfig.CLASSPATH_FILES)); @@ -116,17 +126,109 @@ public void testAddFileToClassPath() throws Exception { // Expected } - DistributedCache.addFileToClassPath(new Path("file:///a"), conf, fs, false); + DistributedCache.addFileToClassPath( + new Path("file:///a"), conf, fs, false); assertEquals("The mapreduce.job.classpath.files property was not " - + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES)); + + "set correctly", "file:/a", + conf.get(MRJobConfig.CLASSPATH_FILES)); assertEquals("The mapreduce.job.cache.files property was not set " - + "correctly", "", conf.get(MRJobConfig.CACHE_FILES, "")); + + "correctly", "", + conf.get(MRJobConfig.CACHE_FILES, "")); - DistributedCache.addFileToClassPath(new Path("file:///b"), conf, fs, false); + DistributedCache.addFileToClassPath( + new Path("file:///b"), conf, fs, false); assertEquals("The mapreduce.job.classpath.files property was not " + "set correctly", "file:/a,file:/b", conf.get(MRJobConfig.CLASSPATH_FILES)); assertEquals("The mapreduce.job.cache.files property was not set " - + "correctly", "", conf.get(MRJobConfig.CACHE_FILES, "")); + + "correctly", "", + conf.get(MRJobConfig.CACHE_FILES, "")); + } + + @Test + public void testGetFileVisibilities() { + Configuration conf = new Configuration(); + StringBuilder sb = new StringBuilder(); + for (LocalResourceVisibility vis : LocalResourceVisibility.values()) { + if (sb.length() != 0) { + sb.append(","); + } + sb.append(vis); + } + conf.set(MRJobConfig.CACHE_FILE_VISIBILITIES, sb.toString()); + LocalResourceVisibility[] visibilityArr = + DistributedCache.getFileVisibilities(conf); + assertEquals("Files resource Visibility Length mismatch", + LocalResourceVisibility.values().length, visibilityArr.length); + int i = 0; + for (LocalResourceVisibility vis : LocalResourceVisibility.values()) { + assertTrue("Vis mis match", + vis.name().equals(visibilityArr[i++].name())); + } + } + + @Test + public void testGetArchiveVisibilities(){ + Configuration conf = new Configuration(); + StringBuilder sb = new StringBuilder(); + for (LocalResourceVisibility vis : LocalResourceVisibility.values()) { + if (sb.length() != 0) { + sb.append(","); + } + sb.append(vis); + } + conf.set(MRJobConfig.CACHE_ARCHIVES_VISIBILITIES, sb.toString()); + LocalResourceVisibility[] visibilityArr = + DistributedCache.getArchiveVisibilities(conf); + assertEquals("Archives resource Visibility Length mismatch", + LocalResourceVisibility.values().length, visibilityArr.length); + int i = 0; + for (LocalResourceVisibility vis : LocalResourceVisibility.values()) { + assertTrue("Vis mis match", + vis.name().equals(visibilityArr[i++].name())); + } + } + + @Test + public void testAddCacheFileWithMeta() throws Exception { + Configuration conf = new Configuration(); + URI uri = new URI("hdfs://namenode/file1"); + DistributedCache.addCacheFileWithMeta( + uri, LocalResourceVisibility.PUBLIC, 1000, 0, conf); + LocalResourceVisibility[] vis = DistributedCache.getFileVisibilities(conf); + assertEquals("Got more than 1 vis", 1, vis.length); + assertTrue("Wrong vis", + vis[0].name().equals(LocalResourceVisibility.PUBLIC.name())); + long[] timeArr = DistributedCache.getFileTimestamps(conf); + assertEquals( + "Got more than 1 timestamp", 1, timeArr.length); + assertEquals("Wrong timestamp", 1000, timeArr[0]); + URI[] archivesArr = DistributedCache.getCacheFiles(conf); + assertEquals( + "Got more than 1 archive", 1, archivesArr.length); + assertTrue( + "Wrong archive", archivesArr[0].toString().equals(uri.toString())); + } + + @Test + public void testAddCacheArchiveWithMeta() throws Exception { + Configuration conf = new Configuration(); + URI uri = new URI("hdfs://namenode/archive1"); + DistributedCache.addCacheArchiveWithMeta( + uri, LocalResourceVisibility.PUBLIC, 1000, 0, conf); + LocalResourceVisibility[] vis = + DistributedCache.getArchiveVisibilities(conf); + assertEquals("Got more than 1 vis", 1, vis.length); + assertTrue("Wrong vis", + vis[0].name().equals(LocalResourceVisibility.PUBLIC.name())); + long[] timeArr = DistributedCache.getArchiveTimestamps(conf); + assertEquals( + "Got more than 1 timestamp", 1, timeArr.length); + assertEquals("Wrong timestamp", 0, timeArr[0]); + URI[] archivesArr = DistributedCache.getCacheArchives(conf); + assertEquals( + "Got more than 1 archive", 1, archivesArr.length); + assertTrue( + "Wrong archive", archivesArr[0].toString().equals(uri.toString())); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/util/TestMRResourceUtil.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/util/TestMRResourceUtil.java new file mode 100644 index 0000000..7051f3b --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/util/TestMRResourceUtil.java @@ -0,0 +1,160 @@ +/** + * 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.mapreduce.util; + +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.MRResource; +import org.apache.hadoop.mapreduce.MRResourceType; +import org.apache.hadoop.mapreduce.MRResourceVisibility; +import org.apache.hadoop.util.GenericOptionsParser; +import org.apache.hadoop.util.ResourceWithVisibilitySetting; +import org.junit.Test; + +/** + * Test class for MRResourceUtil. + */ +public class TestMRResourceUtil { + + @Test + public void testGetJobJarWithDefaultVis() { + // Default visibility + String jobJarPathStr = "/jobjar"; + Configuration conf = new Configuration(); + conf.set(JobContext.JAR, jobJarPathStr); + MRResource jobJar = MRResourceUtil.getJobJar(conf); + assertMRResource( + MRResourceType.JOBJAR, jobJarPathStr, + MRJobConfig.JOBJAR_VISIBILITY_DEFAULT, jobJar); + } + + @Test + public void testGetJobJarWithCustomVis() { + // Default visibility + String jobJarPathStr = "/jobjar1"; + Configuration conf = new Configuration(); + conf.set(JobContext.JAR, jobJarPathStr); + conf.set( + MRJobConfig.JOBJAR_VISIBILITY, MRResourceVisibility.PRIVATE.name()); + MRResource jobJar = MRResourceUtil.getJobJar(conf); + assertMRResource( + MRResourceType.JOBJAR, jobJarPathStr, + MRResourceVisibility.PRIVATE, jobJar); + } + + @Test + public void testGetResourceFromMRConfigLibJar() { + Configuration conf = new Configuration(); + String path0 = "jar0"; + appendToConfig(conf, GenericOptionsParser.TMP_LIBJARS_CONF_KEY, path0); + String path1 = "jar1"; + MRResourceVisibility path1Vis = MRResourceVisibility.PUBLIC; + appendToConfig(conf, GenericOptionsParser.TMP_LIBJARS_CONF_KEY, + path1 + + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + path1Vis.name()); + String path2 = "jar2"; + MRResourceVisibility path2Vis = MRResourceVisibility.PRIVATE; + appendToConfig(conf, MRJobConfig.FILES_FOR_CLASSPATH_AND_SHARED_CACHE, + path2 + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + path2Vis.name()); + List resourceList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.LIBJAR, conf); + assertEquals(3, resourceList.size()); + assertMRResource(MRResourceType.LIBJAR, path0, + MRJobConfig.LIBJARS_VISIBILITY_DEFAULT, resourceList.get(0)); + assertMRResource( + MRResourceType.LIBJAR, path1, path1Vis, resourceList.get(1)); + assertMRResource( + MRResourceType.LIBJAR, path2, path2Vis, resourceList.get(2)); + } + + @Test + public void testGetResourceFromMRConfigFiles() { + Configuration conf = new Configuration(); + String path0 = "file0"; + appendToConfig(conf, GenericOptionsParser.TMP_FILES_CONF_KEY, path0); + String path1 = "file1"; + MRResourceVisibility path1Vis = MRResourceVisibility.PRIVATE; + appendToConfig(conf, GenericOptionsParser.TMP_FILES_CONF_KEY, + path1 + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + path1Vis.name()); + String path2 = "file2"; + MRResourceVisibility path2Vis = MRResourceVisibility.APPLICATION; + appendToConfig(conf, GenericOptionsParser.TMP_FILES_CONF_KEY, + path2 + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + path2Vis.name()); + List resourceList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.FILE, conf); + assertEquals(3, resourceList.size()); + assertMRResource(MRResourceType.FILE, path0, + MRJobConfig.FILES_VISIBILITY_DEFAULT, resourceList.get(0)); + assertMRResource(MRResourceType.FILE, path1, path1Vis, resourceList.get(1)); + assertMRResource(MRResourceType.FILE, path2, path2Vis, resourceList.get(2)); + } + + @Test + public void testGetResourceFromMRConfigArchives() { + Configuration conf = new Configuration(); + String path0 = "archive0"; + appendToConfig(conf, GenericOptionsParser.TMP_ARCHIVES_CONF_KEY, path0); + String path1 = "archive1"; + MRResourceVisibility path1Vis = MRResourceVisibility.PUBLIC; + appendToConfig(conf, GenericOptionsParser.TMP_ARCHIVES_CONF_KEY, + path1 + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + path1Vis.name()); + String path2 = "archive2"; + MRResourceVisibility path2Vis = MRResourceVisibility.APPLICATION; + appendToConfig(conf, GenericOptionsParser.TMP_ARCHIVES_CONF_KEY, + path2 + ResourceWithVisibilitySetting.PATH_SETTING_DELIMITER + + path2Vis.name()); + List resourceList = + MRResourceUtil.getResourceFromMRConfig(MRResourceType.ARCHIVE, conf); + assertEquals(3, resourceList.size()); + assertMRResource(MRResourceType.ARCHIVE, path0, + MRJobConfig.ARCHIEVES_VISIBILITY_DEFAULT, resourceList.get(0)); + assertMRResource( + MRResourceType.ARCHIVE, path1, path1Vis, resourceList.get(1)); + assertMRResource( + MRResourceType.ARCHIVE, path2, path2Vis, resourceList.get(2)); + } + + private void appendToConfig( + Configuration conf, String key, String valToAppend) { + String val = conf.get(key); + val = val == null ? valToAppend : val + "," + valToAppend; + conf.set(key, val); + } + + private void assertMRResource( + MRResourceType expectType, String expectPathStr, + MRResourceVisibility expectVis, MRResource resource) { + assertTrue(expectPathStr.equals(resource.getResourcePathStr())); + assertTrue(expectType.name().equals( + resource.getResourceType().name())); + assertTrue(expectVis.name().equals( + resource.getResourceVisibility().name())); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java index 7d33ed2..bdcc629 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java @@ -62,6 +62,7 @@ import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.protocol.ClientProtocol; import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier; +import org.apache.hadoop.mapreduce.util.MRResourceUtil; import org.apache.hadoop.mapreduce.v2.LogParams; import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol; import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenRequest; @@ -406,10 +407,7 @@ private LocalResource createApplicationResource(FileContext fs, Path p, FileContext fccc = FileContext.getFileContext(jobJarPath.toUri(), jobConf); LocalResourceVisibility jobJarViz = - jobConf.getBoolean(MRJobConfig.JOBJAR_VISIBILITY, - MRJobConfig.JOBJAR_VISIBILITY_DEFAULT) - ? LocalResourceVisibility.PUBLIC - : LocalResourceVisibility.APPLICATION; + MRResourceUtil.getJobJarYarnLocalVisibility(conf); LocalResource rc = createApplicationResource( FileContext.getFileContext(jobJarPath.toUri(), jobConf), jobJarPath, MRJobConfig.JOB_JAR, LocalResourceType.PATTERN, jobJarViz, diff --git a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java index 56f67e5..26a9c5d 100644 --- a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java +++ b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java @@ -34,6 +34,7 @@ import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.MRResourceVisibility; import org.apache.hadoop.tools.rumen.JobStory; import org.apache.hadoop.tools.rumen.JobStoryProducer; import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants; @@ -329,9 +330,9 @@ void updateHDFSDistCacheFilesList(JobStory jobdesc) throws IOException { for (int i = 0; i < files.length; i++) { // Check if visibilities are available because older hadoop versions // didn't have public, private Distributed Caches separately. - boolean visibility = - (visibilities == null) || Boolean.parseBoolean(visibilities[i]); - if (isLocalDistCacheFile(files[i], user, visibility)) { + boolean isPublic = isCacheFilePublic( + visibilities == null ? null : visibilities[i]); + if (isLocalDistCacheFile(files[i], user, isPublic)) { // local FS based distributed cache file. // Create this file on the pseudo local FS on the fly (i.e. when the // simulated job is submitted). @@ -339,7 +340,7 @@ void updateHDFSDistCacheFilesList(JobStory jobdesc) throws IOException { } // distributed cache file on hdfs String mappedPath = mapDistCacheFilePath(files[i], timeStamps[i], - visibility, user); + isPublic, user); // No need to add a distributed cache file path to the list if // (1) the mapped path is already there in the list OR @@ -363,13 +364,14 @@ void updateHDFSDistCacheFilesList(JobStory jobdesc) throws IOException { * @param filePath path of the distributed cache file * @param user job submitter of the job for which <filePath> is a * distributed cache file - * @param visibility true for public distributed cache file + * @param isPublic true for public distributed cache file * @return true if the path provided is of a local file system based * distributed cache file */ static boolean isLocalDistCacheFile(String filePath, String user, - boolean visibility) { - return (!visibility && filePath.contains(user + "/.staging")); + boolean isPublic) { + return (!isPublic + && filePath.contains(user + "/.staging")); } /** @@ -513,9 +515,9 @@ void configureDistCacheFiles(Configuration conf, JobConf jobConf) for (int i = 0; i < files.length; i++) { // Check if visibilities are available because older hadoop versions // didn't have public, private Distributed Caches separately. - boolean visibility = - (visibilities == null) || Boolean.parseBoolean(visibilities[i]); - if (isLocalDistCacheFile(files[i], user, visibility)) { + boolean isPublic = isCacheFilePublic( + visibilities == null ? null : visibilities[i]); + if (isLocalDistCacheFile(files[i], user, isPublic)) { // local FS based distributed cache file. // Create this file on the pseudo local FS. String fileId = MD5Hash.digest(files[i] + timeStamps[i]).toString(); @@ -530,7 +532,7 @@ void configureDistCacheFiles(Configuration conf, JobConf jobConf) // hdfs based distributed cache file. // Get the mapped HDFS path on simulated cluster String mappedPath = mapDistCacheFilePath(files[i], timeStamps[i], - visibility, user); + isPublic, user); cacheFiles.add(mappedPath); } } @@ -547,4 +549,19 @@ void configureDistCacheFiles(Configuration conf, JobConf jobConf) } } } + + boolean isCacheFilePublic(String visibility) { + boolean isPublic; + if (visibility == null) { + return true; + } else if (Boolean.parseBoolean(visibility)) { + // visibilities used to be boolean and the value is true + return true; + } else { + // Could be false or new type of visibility settings with values defined + // as one of {Public, Private, Application} + return MRResourceVisibility.getVisibility(visibility) + == MRResourceVisibility.PUBLIC; + } + } } diff --git a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java index caf0564..ab36dbd 100644 --- a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java +++ b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java @@ -162,8 +162,10 @@ private void validateDistCacheFiles(List filesSizesExpected, Path distCach String[] fileSizes = { "400", "2500", "700", "1200", "1500", "500" }; - String[] visibilities = { "true", "false", "false", "true", "true", "false" }; - String[] timeStamps = { "1234", "2345", "34567", "5434", "125", "134" }; + String[] visibilities = { + "PUBLIC", "APPLICATION", "APPLICATION", + "PUBLIC", "PUBLIC", "APPLICATION" }; + String[] timeStamps = {"1234", "2345", "34567", "5434", "125", "134" }; // DistributedCache.setCacheFiles(fileCaches, conf); conf.setStrings(MRJobConfig.CACHE_FILES, distCacheFiles); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java index d2e8e22..3bd5f39 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java @@ -199,6 +199,7 @@ public String toString() { sb.append(getPath().toString()).append(", "); sb.append(getTimestamp()).append(", "); sb.append(getType()).append(", "); + sb.append(getVisibility()).append(", "); sb.append(getPattern()).append(" }"); return sb.toString(); }