You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by bh...@apache.org on 2018/07/09 20:19:47 UTC

[11/50] [abbrv] hadoop git commit: HADOOP-15571. Multiple FileContexts created with the same configuration object should be allowed to have different umask. Contributed by Vinod Kumar Vavilapalli.

HADOOP-15571. Multiple FileContexts created with the same configuration object should be allowed to have different umask. Contributed by Vinod Kumar Vavilapalli.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/498e3bfb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/498e3bfb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/498e3bfb

Branch: refs/heads/HDDS-48
Commit: 498e3bfb6b93bf542e5581d83e64e920983fe87e
Parents: a129e3e
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Jul 5 14:19:05 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Fri Jul 6 11:56:09 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileContext.java  |  9 ++--
 .../org/apache/hadoop/fs/TestFileContext.java   | 44 +++++++++++++++++++-
 .../logaggregation/AggregatedLogFormat.java     |  6 +--
 3 files changed, 49 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/498e3bfb/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index 5215c3c..0b3889b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@ -219,10 +219,12 @@ public class FileContext {
    * The FileContext is defined by.
    *  1) defaultFS (slash)
    *  2) wd
-   *  3) umask (Obtained by FsPermission.getUMask(conf))
+   *  3) umask (explicitly set via setUMask(),
+   *      falling back to FsPermission.getUMask(conf))
    */   
   private final AbstractFileSystem defaultFS; //default FS for this FileContext.
   private Path workingDir;          // Fully qualified
+  private FsPermission umask;
   private final Configuration conf;
   private final UserGroupInformation ugi;
   final boolean resolveSymlinks;
@@ -575,7 +577,7 @@ public class FileContext {
    * @return the umask of this FileContext
    */
   public FsPermission getUMask() {
-    return FsPermission.getUMask(conf);
+    return (umask != null ? umask : FsPermission.getUMask(conf));
   }
   
   /**
@@ -583,10 +585,9 @@ public class FileContext {
    * @param newUmask  the new umask
    */
   public void setUMask(final FsPermission newUmask) {
-    FsPermission.setUMask(conf, newUmask);
+    this.umask = newUmask;
   }
   
-  
   /**
    * Resolve the path following any symlinks or mount points
    * @param f to be resolved

http://git-wip-us.apache.org/repos/asf/hadoop/blob/498e3bfb/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContext.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContext.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContext.java
index f5fb06f..60b24c7 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContext.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContext.java
@@ -17,13 +17,17 @@
  */
 package org.apache.hadoop.fs;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.net.URI;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.junit.Assert.fail;
-
 public class TestFileContext {
   private static final Logger LOG = LoggerFactory.getLogger(TestFileContext
       .class);
@@ -39,4 +43,40 @@ public class TestFileContext {
       LOG.info("Expected exception: ", ufse);
     }
   }
+
+  @Test
+  public void testConfBasedAndAPIBasedSetUMask() throws Exception {
+
+    Configuration conf = new Configuration();
+
+    String defaultlUMask =
+        conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
+    assertEquals("Default UMask changed!", "022", defaultlUMask);
+
+    URI uri1 = new URI("file://mydfs:50070/");
+    URI uri2 = new URI("file://tmp");
+
+    FileContext fc1 = FileContext.getFileContext(uri1, conf);
+    FileContext fc2 = FileContext.getFileContext(uri2, conf);
+    assertEquals("Umask for fc1 is incorrect", 022, fc1.getUMask().toShort());
+    assertEquals("Umask for fc2 is incorrect", 022, fc2.getUMask().toShort());
+
+    // Till a user explicitly calls FileContext.setUMask(), the updates through
+    // configuration should be reflected..
+    conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "011");
+    assertEquals("Umask for fc1 is incorrect", 011, fc1.getUMask().toShort());
+    assertEquals("Umask for fc2 is incorrect", 011, fc2.getUMask().toShort());
+
+    // Stop reflecting the conf update for specific FileContexts, once an
+    // explicit setUMask is done.
+    conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "066");
+    fc1.setUMask(FsPermission.createImmutable((short) 00033));
+    assertEquals("Umask for fc1 is incorrect", 033, fc1.getUMask().toShort());
+    assertEquals("Umask for fc2 is incorrect", 066, fc2.getUMask().toShort());
+
+    conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
+    fc2.setUMask(FsPermission.createImmutable((short) 00044));
+    assertEquals("Umask for fc1 is incorrect", 033, fc1.getUMask().toShort());
+    assertEquals("Umask for fc2 is incorrect", 044, fc2.getUMask().toShort());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/498e3bfb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 81d5053..4ee5c8a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -96,9 +96,6 @@ public class AggregatedLogFormat {
    */
   private static final FsPermission APP_LOG_FILE_UMASK = FsPermission
       .createImmutable((short) (0640 ^ 0777));
-  /** Default permission for the log file. */
-  private static final FsPermission APP_LOG_FILE_PERM =
-      FsPermission.getFileDefault().applyUMask(APP_LOG_FILE_UMASK);
 
   static {
     RESERVED_KEYS = new HashMap<String, AggregatedLogFormat.LogKey>();
@@ -477,10 +474,11 @@ public class AggregatedLogFormat {
               @Override
               public FSDataOutputStream run() throws Exception {
                 fc = FileContext.getFileContext(remoteAppLogFile.toUri(), conf);
+                fc.setUMask(APP_LOG_FILE_UMASK);
                 return fc.create(
                     remoteAppLogFile,
                     EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
-                    Options.CreateOpts.perms(APP_LOG_FILE_PERM));
+                    new Options.CreateOpts[] {});
               }
             });
       } catch (InterruptedException e) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org