Переглянути джерело

Merging r1541342 through r1541617 from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1541620 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 роки тому
батько
коміт
bf9f8511fa

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -423,6 +423,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10088. copy-nativedistlibs.sh needs to quote snappy lib dir.
     HADOOP-10088. copy-nativedistlibs.sh needs to quote snappy lib dir.
     (Raja Aluri via cnauroth)
     (Raja Aluri via cnauroth)
 
 
+    HADOOP-10093. hadoop-env.cmd sets HADOOP_CLIENT_OPTS with a max heap size
+    that is too small. (Shanyu Zhao via cnauroth)
+
 Release 2.2.1 - UNRELEASED
 Release 2.2.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.cmd

@@ -58,7 +58,7 @@ set HADOOP_DATANODE_OPTS=-Dhadoop.security.logger=ERROR,RFAS %HADOOP_DATANODE_OP
 set HADOOP_SECONDARYNAMENODE_OPTS=-Dhadoop.security.logger=%HADOOP_SECURITY_LOGGER% -Dhdfs.audit.logger=%HDFS_AUDIT_LOGGER% %HADOOP_SECONDARYNAMENODE_OPTS%
 set HADOOP_SECONDARYNAMENODE_OPTS=-Dhadoop.security.logger=%HADOOP_SECURITY_LOGGER% -Dhdfs.audit.logger=%HDFS_AUDIT_LOGGER% %HADOOP_SECONDARYNAMENODE_OPTS%
 
 
 @rem The following applies to multiple commands (fs, dfs, fsck, distcp etc)
 @rem The following applies to multiple commands (fs, dfs, fsck, distcp etc)
-set HADOOP_CLIENT_OPTS=-Xmx128m %HADOOP_CLIENT_OPTS%
+set HADOOP_CLIENT_OPTS=-Xmx512m %HADOOP_CLIENT_OPTS%
 @rem set HADOOP_JAVA_PLATFORM_OPTS="-XX:-UsePerfData %HADOOP_JAVA_PLATFORM_OPTS%"
 @rem set HADOOP_JAVA_PLATFORM_OPTS="-XX:-UsePerfData %HADOOP_JAVA_PLATFORM_OPTS%"
 
 
 @rem On secure datanodes, user to run the datanode as after dropping privileges
 @rem On secure datanodes, user to run the datanode as after dropping privileges

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -192,6 +192,8 @@ Trunk (Unreleased)
 
 
     HDFS-5450. Better API for getting the cached blocks locations. (wang)
     HDFS-5450. Better API for getting the cached blocks locations. (wang)
 
 
+    HDFS-5485. Add command-line support for modifyDirective. (cmccabe)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
 

+ 82 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java

@@ -248,6 +248,84 @@ public class CacheAdmin extends Configured implements Tool {
     }
     }
   }
   }
 
 
+  private static class ModifyPathBasedCacheDirectiveCommand implements Command {
+    @Override
+    public String getName() {
+      return "-modifyDirective";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() +
+          " -id <id> [-path <path>] [-replication <replication>] " +
+          "[-pool <pool-name>] ]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = getOptionDescriptionListing();
+      listing.addRow("<id>", "The ID of the directive to modify (required)");
+      listing.addRow("<path>", "A path to cache. The path can be " +
+          "a directory or a file. (optional)");
+      listing.addRow("<replication>", "The cache replication factor to use. " +
+          "(optional)");
+      listing.addRow("<pool-name>", "The pool to which the directive will be " +
+          "added. You must have write permission on the cache pool "
+          + "in order to move a directive into it. (optional)");
+      return getShortUsage() + "\n" +
+        "Modify a PathBasedCache directive.\n\n" +
+        listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      PathBasedCacheDirective.Builder builder =
+        new PathBasedCacheDirective.Builder();
+      boolean modified = false;
+      String idString = StringUtils.popOptionWithArgument("-id", args);
+      if (idString == null) {
+        System.err.println("You must specify a directive ID with -id.");
+        return 1;
+      }
+      builder.setId(Long.parseLong(idString));
+      String path = StringUtils.popOptionWithArgument("-path", args);
+      if (path != null) {
+        builder.setPath(new Path(path));
+        modified = true;
+      }
+      String replicationString =
+        StringUtils.popOptionWithArgument("-replication", args);
+      if (replicationString != null) {
+        builder.setReplication(Short.parseShort(replicationString));
+        modified = true;
+      }
+      String poolName =
+        StringUtils.popOptionWithArgument("-pool", args);
+      if (poolName != null) {
+        builder.setPool(poolName);
+        modified = true;
+      }
+      if (!args.isEmpty()) {
+        System.err.println("Can't understand argument: " + args.get(0));
+        System.err.println("Usage is " + getShortUsage());
+        return 1;
+      }
+      if (!modified) {
+        System.err.println("No modifications were specified.");
+        return 1;
+      }
+      DistributedFileSystem dfs = getDFS(conf);
+      try {
+        dfs.modifyPathBasedCacheDirective(builder.build());
+        System.out.println("Modified PathBasedCache entry " + idString);
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
   private static class RemovePathBasedCacheDirectivesCommand implements Command {
   private static class RemovePathBasedCacheDirectivesCommand implements Command {
     @Override
     @Override
     public String getName() {
     public String getName() {
@@ -352,6 +430,7 @@ public class CacheAdmin extends Configured implements Tool {
       TableListing tableListing = new TableListing.Builder().
       TableListing tableListing = new TableListing.Builder().
           addField("ID", Justification.LEFT).
           addField("ID", Justification.LEFT).
           addField("POOL", Justification.LEFT).
           addField("POOL", Justification.LEFT).
+          addField("REPLICATION", Justification.LEFT).
           addField("PATH", Justification.LEFT).
           addField("PATH", Justification.LEFT).
           build();
           build();
       DistributedFileSystem dfs = getDFS(conf);
       DistributedFileSystem dfs = getDFS(conf);
@@ -362,6 +441,7 @@ public class CacheAdmin extends Configured implements Tool {
         PathBasedCacheDirective directive = iter.next();
         PathBasedCacheDirective directive = iter.next();
         String row[] = new String[] {
         String row[] = new String[] {
             "" + directive.getId(), directive.getPool(),
             "" + directive.getId(), directive.getPool(),
+            "" + directive.getReplication(),
             directive.getPath().toUri().getPath(),
             directive.getPath().toUri().getPath(),
         };
         };
         tableListing.addRow(row);
         tableListing.addRow(row);
@@ -744,9 +824,10 @@ public class CacheAdmin extends Configured implements Tool {
 
 
   private static Command[] COMMANDS = {
   private static Command[] COMMANDS = {
     new AddPathBasedCacheDirectiveCommand(),
     new AddPathBasedCacheDirectiveCommand(),
+    new ModifyPathBasedCacheDirectiveCommand(),
+    new ListPathBasedCacheDirectiveCommand(),
     new RemovePathBasedCacheDirectiveCommand(),
     new RemovePathBasedCacheDirectiveCommand(),
     new RemovePathBasedCacheDirectivesCommand(),
     new RemovePathBasedCacheDirectivesCommand(),
-    new ListPathBasedCacheDirectiveCommand(),
     new AddCachePoolCommand(),
     new AddCachePoolCommand(),
     new ModifyCachePoolCommand(),
     new ModifyCachePoolCommand(),
     new RemoveCachePoolCommand(),
     new RemoveCachePoolCommand(),

+ 37 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml

@@ -180,15 +180,15 @@
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>1   pool1  /foo</expected-output>
+          <expected-output>1   pool1  1            /foo</expected-output>
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>2   pool1  /bar</expected-output>
+          <expected-output>2   pool1  1            /bar</expected-output>
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>3   pool1  /baz</expected-output>
+          <expected-output>3   pool1  2            /baz</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
@@ -234,11 +234,11 @@
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>8   pool2  /baz</expected-output>
+          <expected-output>8   pool2  1            /baz</expected-output>
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>9   pool2  /buz</expected-output>
+          <expected-output>9   pool2  1            /buz</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
@@ -265,11 +265,11 @@
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>10  pool1  /foo</expected-output>
+          <expected-output>10  pool1  1            /foo</expected-output>
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>12  pool2  /foo</expected-output>
+          <expected-output>12  pool2  1            /foo</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
@@ -296,7 +296,7 @@
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>16  pool2  /foo</expected-output>
+          <expected-output>16  pool2  1            /foo</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
@@ -320,7 +320,7 @@
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>19  pool1  /bar</expected-output>
+          <expected-output>19  pool1  1            /bar</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
@@ -349,11 +349,37 @@
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>22  pool1  /bar</expected-output>
+          <expected-output>22  pool1  1            /bar</expected-output>
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>24  pool2  /bar</expected-output>
+          <expected-output>24  pool2  1            /bar</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!--Tested -->
+      <description>Testing modifying directives</description>
+      <test-commands>
+        <cache-admin-command>-addPool pool1</cache-admin-command>
+        <cache-admin-command>-addPool pool2</cache-admin-command>
+        <cache-admin-command>-addDirective -path /foo -pool pool2</cache-admin-command>
+        <cache-admin-command>-modifyDirective -id 25 -path /bar2</cache-admin-command>
+        <cache-admin-command>-modifyDirective -id 25 -pool pool1 -path /bar3</cache-admin-command>
+        <cache-admin-command>-listDirectives -path /bar3</cache-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <cache-admin-command>-removePool pool1</cache-admin-command>
+        <cache-admin-command>-removePool pool2</cache-admin-command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Found 1 entry</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>25  pool1  1            /bar3</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>

+ 5 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -167,6 +167,9 @@ Release 2.3.0 - UNRELEASED
     MAPREDUCE-1176. FixedLengthInputFormat and FixedLengthRecordReader
     MAPREDUCE-1176. FixedLengthInputFormat and FixedLengthRecordReader
     (Mariappan Asokan and BitsOfInfo via Sandy Ryza)
     (Mariappan Asokan and BitsOfInfo via Sandy Ryza)
 
 
+    MAPREDUCE-5613. DefaultSpeculator holds and checks hashmap that is always
+    empty (Gera Shegalov via Sandy Ryza)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
@@ -204,6 +207,8 @@ Release 2.3.0 - UNRELEASED
     created by CombineFileInputFormat to fail (Robert Parker and Jason Lowe
     created by CombineFileInputFormat to fail (Robert Parker and Jason Lowe
     via jlowe)
     via jlowe)
 
 
+    MAPREDUCE-5610. TestSleepJob fails in jdk7 (Jonathan Eagles via jlowe)
+
 Release 2.2.1 - UNRELEASED
 Release 2.2.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 0 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java

@@ -28,7 +28,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicInteger;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -75,9 +74,6 @@ public class DefaultSpeculator extends AbstractService implements
   private final ConcurrentMap<TaskId, Boolean> runningTasks
   private final ConcurrentMap<TaskId, Boolean> runningTasks
       = new ConcurrentHashMap<TaskId, Boolean>();
       = new ConcurrentHashMap<TaskId, Boolean>();
 
 
-  private final Map<Task, AtomicBoolean> pendingSpeculations
-      = new ConcurrentHashMap<Task, AtomicBoolean>();
-
   // Used to track any TaskAttempts that aren't heart-beating for a while, so
   // Used to track any TaskAttempts that aren't heart-beating for a while, so
   // that we can aggressively speculate instead of waiting for task-timeout.
   // that we can aggressively speculate instead of waiting for task-timeout.
   private final ConcurrentMap<TaskAttemptId, TaskAttemptHistoryStatistics>
   private final ConcurrentMap<TaskAttemptId, TaskAttemptHistoryStatistics>
@@ -328,13 +324,6 @@ public class DefaultSpeculator extends AbstractService implements
 
 
     estimator.updateAttempt(reportedStatus, timestamp);
     estimator.updateAttempt(reportedStatus, timestamp);
 
 
-    // If the task is already known to be speculation-bait, don't do anything
-    if (pendingSpeculations.get(task) != null) {
-      if (pendingSpeculations.get(task).get()) {
-        return;
-      }
-    }
-
     if (stateString.equals(TaskAttemptState.RUNNING.name())) {
     if (stateString.equals(TaskAttemptState.RUNNING.name())) {
       runningTasks.putIfAbsent(taskID, Boolean.TRUE);
       runningTasks.putIfAbsent(taskID, Boolean.TRUE);
     } else {
     } else {

+ 2 - 2
hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/JobCreator.java

@@ -63,8 +63,8 @@ public enum JobCreator {
       Configuration conf, long submissionMillis, JobStory jobdesc, Path outRoot,
       Configuration conf, long submissionMillis, JobStory jobdesc, Path outRoot,
       UserGroupInformation ugi, int seq) throws IOException {
       UserGroupInformation ugi, int seq) throws IOException {
       int numLocations = conf.getInt(SLEEPJOB_RANDOM_LOCATIONS, 0);
       int numLocations = conf.getInt(SLEEPJOB_RANDOM_LOCATIONS, 0);
-      if (numLocations < 0) numLocations=0;
-      if ((numLocations > 0) && (hosts == null)) {
+      if (numLocations < 0) numLocations = 0;
+      if (hosts == null) {
         final JobClient client = new JobClient(new JobConf(conf));
         final JobClient client = new JobClient(new JobConf(conf));
         ClusterStatus stat = client.getClusterStatus(true);
         ClusterStatus stat = client.getClusterStatus(true);
         final int nTrackers = stat.getTaskTrackers();
         final int nTrackers = stat.getTaskTrackers();

+ 11 - 11
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestSleepJob.java

@@ -58,17 +58,6 @@ public class TestSleepJob extends CommonJobTest {
   }
   }
 
 
 
 
-  /*
-  * test RandomLocation
-  */
-  @Test
-  public void testRandomLocation() throws Exception {
-    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
-
-    testRandomLocation(1, 10, ugi);
-    testRandomLocation(2, 10, ugi);
-  }
-
   @Test
   @Test
   public void testMapTasksOnlySleepJobs() throws Exception {
   public void testMapTasksOnlySleepJobs() throws Exception {
     Configuration configuration = GridmixTestUtils.mrvl.getConfig();
     Configuration configuration = GridmixTestUtils.mrvl.getConfig();
@@ -90,6 +79,17 @@ public class TestSleepJob extends CommonJobTest {
     assertEquals(6, seq);
     assertEquals(6, seq);
   }
   }
 
 
+  /*
+  * test RandomLocation
+  */
+  @Test
+  public void testRandomLocation() throws Exception {
+    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+
+    testRandomLocation(1, 10, ugi);
+    testRandomLocation(2, 10, ugi);
+  }
+
   // test Serial submit
   // test Serial submit
   @Test
   @Test
   public void testSerialSubmit() throws Exception {
   public void testSerialSubmit() throws Exception {

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -1636,6 +1636,9 @@ Release 0.23.10 - UNRELEASED
     YARN-1176. RM web services ClusterMetricsInfo total nodes doesn't include 
     YARN-1176. RM web services ClusterMetricsInfo total nodes doesn't include 
     unhealthy nodes (Jonathan Eagles via tgraves)
     unhealthy nodes (Jonathan Eagles via tgraves)
 
 
+    YARN-1386. NodeManager mistakenly loses resources and relocalizes them
+    (Jason Lowe via jeagles)
+
 Release 0.23.9 - 2013-07-08
 Release 0.23.9 - 2013-07-08
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 12 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
@@ -82,6 +83,8 @@ public class ContainerLocalizer {
   public static final String WORKDIR = "work";
   public static final String WORKDIR = "work";
   private static final String APPCACHE_CTXT_FMT = "%s.app.cache.dirs";
   private static final String APPCACHE_CTXT_FMT = "%s.app.cache.dirs";
   private static final String USERCACHE_CTXT_FMT = "%s.user.cache.dirs";
   private static final String USERCACHE_CTXT_FMT = "%s.user.cache.dirs";
+  private static final FsPermission FILECACHE_PERMS =
+      new FsPermission((short)0710);
 
 
   private final String user;
   private final String user;
   private final String appId;
   private final String appId;
@@ -363,16 +366,23 @@ public class ContainerLocalizer {
       // $x/usercache/$user/filecache
       // $x/usercache/$user/filecache
       Path userFileCacheDir = new Path(base, FILECACHE);
       Path userFileCacheDir = new Path(base, FILECACHE);
       usersFileCacheDirs[i] = userFileCacheDir.toString();
       usersFileCacheDirs[i] = userFileCacheDir.toString();
-      lfs.mkdir(userFileCacheDir, null, false);
+      createDir(lfs, userFileCacheDir, FILECACHE_PERMS, false);
       // $x/usercache/$user/appcache/$appId
       // $x/usercache/$user/appcache/$appId
       Path appBase = new Path(base, new Path(APPCACHE, appId));
       Path appBase = new Path(base, new Path(APPCACHE, appId));
       // $x/usercache/$user/appcache/$appId/filecache
       // $x/usercache/$user/appcache/$appId/filecache
       Path appFileCacheDir = new Path(appBase, FILECACHE);
       Path appFileCacheDir = new Path(appBase, FILECACHE);
       appsFileCacheDirs[i] = appFileCacheDir.toString();
       appsFileCacheDirs[i] = appFileCacheDir.toString();
-      lfs.mkdir(appFileCacheDir, null, false);
+      createDir(lfs, appFileCacheDir, FILECACHE_PERMS, false);
     }
     }
     conf.setStrings(String.format(APPCACHE_CTXT_FMT, appId), appsFileCacheDirs);
     conf.setStrings(String.format(APPCACHE_CTXT_FMT, appId), appsFileCacheDirs);
     conf.setStrings(String.format(USERCACHE_CTXT_FMT, user), usersFileCacheDirs);
     conf.setStrings(String.format(USERCACHE_CTXT_FMT, user), usersFileCacheDirs);
   }
   }
 
 
+  private static void createDir(FileContext lfs, Path dirPath,
+      FsPermission perms, boolean createParent) throws IOException {
+    lfs.mkdir(dirPath, perms, createParent);
+    if (!perms.equals(perms.applyUMask(lfs.getUMask()))) {
+      lfs.setPermission(dirPath, perms);
+    }
+  }
 }
 }

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestContainerLocalizer.java

@@ -83,6 +83,7 @@ public class TestContainerLocalizer {
   static final Log LOG = LogFactory.getLog(TestContainerLocalizer.class);
   static final Log LOG = LogFactory.getLog(TestContainerLocalizer.class);
   static final Path basedir =
   static final Path basedir =
       new Path("target", TestContainerLocalizer.class.getName());
       new Path("target", TestContainerLocalizer.class.getName());
+  static final FsPermission CACHE_DIR_PERM = new FsPermission((short)0710);
 
 
   static final String appUser = "yak";
   static final String appUser = "yak";
   static final String appId = "app_RM_0";
   static final String appId = "app_RM_0";
@@ -171,12 +172,12 @@ public class TestContainerLocalizer {
       Path base = new Path(new Path(p, ContainerLocalizer.USERCACHE), appUser);
       Path base = new Path(new Path(p, ContainerLocalizer.USERCACHE), appUser);
       Path privcache = new Path(base, ContainerLocalizer.FILECACHE);
       Path privcache = new Path(base, ContainerLocalizer.FILECACHE);
       // $x/usercache/$user/filecache
       // $x/usercache/$user/filecache
-      verify(spylfs).mkdir(eq(privcache), isA(FsPermission.class), eq(false));
+      verify(spylfs).mkdir(eq(privcache), eq(CACHE_DIR_PERM), eq(false));
       Path appDir =
       Path appDir =
         new Path(base, new Path(ContainerLocalizer.APPCACHE, appId));
         new Path(base, new Path(ContainerLocalizer.APPCACHE, appId));
       // $x/usercache/$user/appcache/$appId/filecache
       // $x/usercache/$user/appcache/$appId/filecache
       Path appcache = new Path(appDir, ContainerLocalizer.FILECACHE);
       Path appcache = new Path(appDir, ContainerLocalizer.FILECACHE);
-      verify(spylfs).mkdir(eq(appcache), isA(FsPermission.class), eq(false));
+      verify(spylfs).mkdir(eq(appcache), eq(CACHE_DIR_PERM), eq(false));
     }
     }
     // verify tokens read at expected location
     // verify tokens read at expected location
     verify(spylfs).open(tokenPath);
     verify(spylfs).open(tokenPath);