ソースを参照

HADOOP-6234. Add new option dfs.umaskmode to set umask in configuration to use octal or symbolic instead of decimal. Contributed by Jakob Homan.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@812652 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 16 年 前
コミット
cdbb724edb

+ 4 - 0
CHANGES.txt

@@ -181,6 +181,10 @@ Trunk (unreleased changes)
     HADOOP-6235. Adds new method to FileSystem for clients to get server
     defaults. (Kan Zhang via suresh)
 
+    HADOOP-6234. Add new option dfs.umaskmode to set umask in configuration
+    to use octal or symbolic instead of decimal. (Jakob Homan via suresh)
+
+
   IMPROVEMENTS
 
     HADOOP-4565. Added CombineFileInputFormat to use data locality information

+ 2 - 2
src/docs/src/documentation/content/xdocs/hdfs_permissions_guide.xml

@@ -178,9 +178,9 @@ If a cluster starts with a version 0.15 data set (<code>fsimage</code>), all fil
 	<dd>
 		The choice of initial mode during upgrade. The <em>x</em> permission is <em>never</em> set for files. For configuration files, the decimal value <em>511<sub>10</sub></em> may be used.
 	</dd>
-	<dt><code>dfs.umask = 022</code></dt>
+	<dt><code>dfs.umaskmode = 022</code></dt>
 	<dd>
-		The <code>umask</code> used when creating files and directories. For configuration files, the decimal value <em>18<sub>10</sub></em> may be used.
+		The <code>umask</code> used when creating files and directories. May be specified either via three octal digits or symbolic values, with the same constraints as the dfs chmod command.
 	</dd>
 </dl>
 </section>

+ 13 - 147
src/java/org/apache/hadoop/fs/FsShellPermissions.java

@@ -23,6 +23,7 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.fs.FsShell.CmdHandler;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.ChmodParser;
 
 
 /**
@@ -39,168 +40,33 @@ class FsShellPermissions {
    * also enforce octal mode specifications of either 3 digits without a sticky
    * bit setting or four digits with a sticky bit setting.
    */
-  private static Pattern chmodNormalPattern =
-   Pattern.compile("\\G\\s*([ugoa]*)([+=-]+)([rwxXt]+)([,\\s]*)\\s*");
-  private static Pattern chmodOctalPattern =
-            Pattern.compile("^\\s*[+]?([01]?)([0-7]{3})\\s*$");
 
   static String CHMOD_USAGE =
                             "-chmod [-R] <MODE[,MODE]... | OCTALMODE> PATH...";
 
+  private static  ChmodParser pp;
+  
   private static class ChmodHandler extends CmdHandler {
 
-    private short userMode;
-    private short groupMode;
-    private short othersMode;
-    private short stickyMode;
-    private char userType = '+';
-    private char groupType = '+';
-    private char othersType = '+';
-    private char stickyBitType = '+';
-
-    private void applyNormalPattern(String modeStr, Matcher matcher)
-                                    throws IOException {
-      // Are there multiple permissions stored in one chmod?
-      boolean commaSeperated = false;
-
-      for(int i=0; i < 1 || matcher.end() < modeStr.length(); i++) {
-        if (i>0 && (!commaSeperated || !matcher.find())) {
-          patternError(modeStr);
-        }
-
-        /* groups : 1 : [ugoa]*
-         *          2 : [+-=]
-         *          3 : [rwxXt]+
-         *          4 : [,\s]*
-         */
-
-        String str = matcher.group(2);
-        char type = str.charAt(str.length() - 1);
-
-        boolean user, group, others, stickyBit;
-        user = group = others = stickyBit = false;
-
-        for(char c : matcher.group(1).toCharArray()) {
-          switch (c) {
-          case 'u' : user = true; break;
-          case 'g' : group = true; break;
-          case 'o' : others = true; break;
-          case 'a' : break;
-          default  : throw new RuntimeException("Unexpected");          
-          }
-        }
-
-        if (!(user || group || others)) { // same as specifying 'a'
-          user = group = others = true;
-        }
-
-        short mode = 0;
-
-        for(char c : matcher.group(3).toCharArray()) {
-          switch (c) {
-          case 'r' : mode |= 4; break;
-          case 'w' : mode |= 2; break;
-          case 'x' : mode |= 1; break;
-          case 'X' : mode |= 8; break;
-          case 't' : stickyBit = true; break;
-          default  : throw new RuntimeException("Unexpected");
-          }
-        }
-
-        if ( user ) {
-          userMode = mode;
-          userType = type;
-        }
-
-        if ( group ) {
-          groupMode = mode;
-          groupType = type;
-        }
-
-        if ( others ) {
-          othersMode = mode;
-          othersType = type;
-          
-          stickyMode = (short) (stickyBit ? 1 : 0);
-          stickyBitType = type;
-        }
-
-        commaSeperated = matcher.group(4).contains(",");
-      }
-    }
-
-    private void applyOctalPattern(String modeStr, Matcher matcher) {
-      userType = groupType = othersType = '=';
-
-      // Check if sticky bit is specified
-      String sb = matcher.group(1);
-      if(!sb.isEmpty()) {
-        stickyMode = Short.valueOf(sb.substring(0, 1));
-        stickyBitType = '=';
-      }
-
-      String str = matcher.group(2);
-      userMode = Short.valueOf(str.substring(0, 1));
-      groupMode = Short.valueOf(str.substring(1, 2));
-      othersMode = Short.valueOf(str.substring(2, 3));      
-    }
-
-    private void patternError(String mode) throws IOException {
-      throw new IOException("chmod : mode '" + mode + 
-                            "' does not match the expected pattern.");      
-    }
-
     ChmodHandler(FileSystem fs, String modeStr) throws IOException {
       super("chmod", fs);
-      Matcher matcher = null;
-
-      if ((matcher = chmodNormalPattern.matcher(modeStr)).find()) {
-        applyNormalPattern(modeStr, matcher);
-      } else if ((matcher = chmodOctalPattern.matcher(modeStr)).matches()) {
-        applyOctalPattern(modeStr, matcher);
-      } else {
-        patternError(modeStr);
+      try {
+        pp = new ChmodParser(modeStr);
+      } catch(IllegalArgumentException iea) {
+        patternError(iea.getMessage());
       }
     }
 
-    private int applyChmod(char type, int mode, int existing, boolean exeOk) {
-      boolean capX = false;
-
-      if ((mode&8) != 0) { // convert X to x;
-        capX = true;
-        mode &= ~8;
-        mode |= 1;
-      }
-
-      switch (type) {
-      case '+' : mode = mode | existing; break;
-      case '-' : mode = (~mode) & existing; break;
-      case '=' : break;
-      default  : throw new RuntimeException("Unexpected");      
-      }
-
-      // if X is specified add 'x' only if exeOk or x was already set.
-      if (capX && !exeOk && (mode&1) != 0 && (existing&1) == 0) {
-        mode &= ~1; // remove x
-      }
-
-      return mode;
+    private void patternError(String mode) throws IOException {
+     throw new IOException("chmod : mode '" + mode + 
+         "' does not match the expected pattern.");      
     }
-
+    
     @Override
     public void run(FileStatus file, FileSystem srcFs) throws IOException {
-      FsPermission perms = file.getPermission();
-      int existing = perms.toShort();
-      boolean exeOk = file.isDir() || (existing & 0111) != 0;
-      int newperms = ( applyChmod(stickyBitType, stickyMode,
-                             (existing>>>9), false) << 9 |
-                       applyChmod(userType, userMode,
-                             (existing>>>6)&7, exeOk) << 6 |
-                       applyChmod(groupType, groupMode,
-                             (existing>>>3)&7, exeOk) << 3 |
-                       applyChmod(othersType, othersMode, existing&7, exeOk));
+      int newperms = pp.applyNewPermission(file);
 
-      if (existing != newperms) {
+      if (file.getPermission().toShort() != newperms) {
         try {
           srcFs.setPermission(file.getPath(), 
                                 new FsPermission((short)newperms));

+ 51 - 0
src/java/org/apache/hadoop/fs/permission/ChmodParser.java

@@ -0,0 +1,51 @@
+/**
+ * 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.fs.permission;
+
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.fs.FileStatus;
+
+/**
+ * Parse a permission mode passed in from a chmod command and apply that
+ * mode against an existing file.
+ */
+public class ChmodParser extends PermissionParser {
+  private static Pattern chmodOctalPattern =
+    Pattern.compile("^\\s*[+]?([01]?)([0-7]{3})\\s*$");
+  private static Pattern chmodNormalPattern =
+    Pattern.compile("\\G\\s*([ugoa]*)([+=-]+)([rwxXt]+)([,\\s]*)\\s*");
+  
+  public ChmodParser(String modeStr) throws IllegalArgumentException {
+    super(modeStr, chmodNormalPattern, chmodOctalPattern);
+  }
+
+  /**
+   * Apply permission against specified file and determine what the
+   * new mode would be
+   * @param file File against which to apply mode
+   * @return File's new mode if applied.
+   */
+  public short applyNewPermission(FileStatus file) {
+    FsPermission perms = file.getPermission();
+    int existing = perms.toShort();
+    boolean exeOk = file.isDir() || (existing & 0111) != 0;
+    
+    return (short)combineModes(existing, exeOk);
+  }
+}

+ 28 - 6
src/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -17,17 +17,23 @@
  */
 package org.apache.hadoop.fs.permission;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.*;
-
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
 /**
  * A class for file/directory permissions.
  */
 public class FsPermission implements Writable {
+  private static final Log LOG = LogFactory.getLog(FsPermission.class);
+  
   static final WritableFactory FACTORY = new WritableFactory() {
     public Writable newInstance() { return new FsPermission(); }
   };
@@ -176,15 +182,31 @@ public class FsPermission implements Writable {
   }
 
   /** umask property label */
-  public static final String UMASK_LABEL = "dfs.umask";
+  public static final String DEPRECATED_UMASK_LABEL = "dfs.umask"; 
+  public static final String UMASK_LABEL = "dfs.umaskmode";
   public static final int DEFAULT_UMASK = 0022;
 
   /** Get the user file creation mask (umask) */
   public static FsPermission getUMask(Configuration conf) {
     int umask = DEFAULT_UMASK;
-    if (conf != null) {
-      umask = conf.getInt(UMASK_LABEL, DEFAULT_UMASK);
+    
+    // Attempt to pull value from configuration, trying new key first and then
+    // deprecated key, along with a warning, if not present
+    if(conf != null) {
+      String confUmask = conf.get(UMASK_LABEL);
+      if(confUmask != null) { // UMASK_LABEL is set
+        umask = new UmaskParser(confUmask).getUMask();
+      } else { // check for deprecated key label
+        int oldStyleValue = conf.getInt(DEPRECATED_UMASK_LABEL, Integer.MIN_VALUE);
+        if(oldStyleValue != Integer.MIN_VALUE) { // Property was set with old key
+          LOG.warn(DEPRECATED_UMASK_LABEL + " configuration key is deprecated. " +
+              "Convert to " + UMASK_LABEL + ", using octal or symbolic umask " +
+              "specifications.");
+          umask = oldStyleValue;
+        }
+      }
     }
+    
     return new FsPermission((short)umask);
   }
 

+ 195 - 0
src/java/org/apache/hadoop/fs/permission/PermissionParser.java

@@ -0,0 +1,195 @@
+/**
+ * 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.fs.permission;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Base class for parsing either chmod permissions or umask permissions.
+ * Includes common code needed by either operation as implemented in
+ * UmaskParser and ChmodParser classes.
+ */
+class PermissionParser {
+  protected short userMode;
+  protected short groupMode;
+  protected short othersMode;
+  protected short stickyMode;
+  protected char userType = '+';
+  protected char groupType = '+';
+  protected char othersType = '+';
+  protected char stickyBitType = '+';
+  
+  /**
+   * Begin parsing permission stored in modeStr
+   * 
+   * @param modeStr Permission mode, either octal or symbolic
+   * @param symbolic Use-case specific symbolic pattern to match against
+   * @throws IllegalArgumentException if unable to parse modeStr
+   */
+  public PermissionParser(String modeStr, Pattern symbolic, Pattern octal) 
+       throws IllegalArgumentException {
+    Matcher matcher = null;
+
+    if ((matcher = symbolic.matcher(modeStr)).find()) {
+      applyNormalPattern(modeStr, matcher);
+    } else if ((matcher = octal.matcher(modeStr)).matches()) {
+      applyOctalPattern(modeStr, matcher);
+    } else {
+      throw new IllegalArgumentException(modeStr);
+    }
+  }
+
+  private void applyNormalPattern(String modeStr, Matcher matcher) {
+    // Are there multiple permissions stored in one chmod?
+    boolean commaSeperated = false;
+
+    for (int i = 0; i < 1 || matcher.end() < modeStr.length(); i++) {
+      if (i > 0 && (!commaSeperated || !matcher.find())) {
+        throw new IllegalArgumentException(modeStr);
+      }
+
+      /*
+       * groups : 1 : [ugoa]* 2 : [+-=] 3 : [rwxXt]+ 4 : [,\s]*
+       */
+
+      String str = matcher.group(2);
+      char type = str.charAt(str.length() - 1);
+
+      boolean user, group, others, stickyBit;
+      user = group = others = stickyBit = false;
+
+      for (char c : matcher.group(1).toCharArray()) {
+        switch (c) {
+        case 'u':
+          user = true;
+          break;
+        case 'g':
+          group = true;
+          break;
+        case 'o':
+          others = true;
+          break;
+        case 'a':
+          break;
+        default:
+          throw new RuntimeException("Unexpected");
+        }
+      }
+
+      if (!(user || group || others)) { // same as specifying 'a'
+        user = group = others = true;
+      }
+
+      short mode = 0;
+
+      for (char c : matcher.group(3).toCharArray()) {
+        switch (c) {
+        case 'r':
+          mode |= 4;
+          break;
+        case 'w':
+          mode |= 2;
+          break;
+        case 'x':
+          mode |= 1;
+          break;
+        case 'X':
+          mode |= 8;
+          break;
+        case 't':
+          stickyBit = true;
+          break;
+        default:
+          throw new RuntimeException("Unexpected");
+        }
+      }
+
+      if (user) {
+        userMode = mode;
+        userType = type;
+      }
+
+      if (group) {
+        groupMode = mode;
+        groupType = type;
+      }
+
+      if (others) {
+        othersMode = mode;
+        othersType = type;
+
+        stickyMode = (short) (stickyBit ? 1 : 0);
+        stickyBitType = type;
+      }
+
+      commaSeperated = matcher.group(4).contains(",");
+    }
+  }
+
+  private void applyOctalPattern(String modeStr, Matcher matcher) {
+    userType = groupType = othersType = '=';
+
+    // Check if sticky bit is specified
+    String sb = matcher.group(1);
+    if (!sb.isEmpty()) {
+      stickyMode = Short.valueOf(sb.substring(0, 1));
+      stickyBitType = '=';
+    }
+
+    String str = matcher.group(2);
+    userMode = Short.valueOf(str.substring(0, 1));
+    groupMode = Short.valueOf(str.substring(1, 2));
+    othersMode = Short.valueOf(str.substring(2, 3));
+  }
+
+  protected int combineModes(int existing, boolean exeOk) {
+    return   combineModeSegments(stickyBitType, stickyMode, 
+                (existing>>>9), false) << 9 |
+             combineModeSegments(userType, userMode,
+                (existing>>>6)&7, exeOk) << 6 |
+             combineModeSegments(groupType, groupMode,
+                (existing>>>3)&7, exeOk) << 3 |
+             combineModeSegments(othersType, othersMode, existing&7, exeOk);
+  }
+  
+  protected int combineModeSegments(char type, int mode, 
+                                    int existing, boolean exeOk) {
+    boolean capX = false;
+
+    if ((mode&8) != 0) { // convert X to x;
+      capX = true;
+      mode &= ~8;
+      mode |= 1;
+    }
+
+    switch (type) {
+    case '+' : mode = mode | existing; break;
+    case '-' : mode = (~mode) & existing; break;
+    case '=' : break;
+    default  : throw new RuntimeException("Unexpected");      
+    }
+
+    // if X is specified add 'x' only if exeOk or x was already set.
+    if (capX && !exeOk && (mode&1) != 0 && (existing&1) == 0) {
+      mode &= ~1; // remove x
+    }
+
+    return mode;
+  }
+}

+ 45 - 0
src/java/org/apache/hadoop/fs/permission/UmaskParser.java

@@ -0,0 +1,45 @@
+/**
+ * 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.fs.permission;
+
+import java.util.regex.Pattern;
+
+/**
+ * Parse umask value provided as a string, either in octal or symbolic
+ * format and return it as a short value. Umask values are slightly
+ * different from standard modes as they cannot specify sticky bit
+ * or X.
+ *
+ */
+class UmaskParser extends PermissionParser {
+  private static Pattern chmodOctalPattern =
+    Pattern.compile("^\\s*[+]?()([0-7]{3})\\s*$"); // no leading 1 for sticky bit
+  private static Pattern umaskSymbolicPattern =    /* not allow X or t */
+    Pattern.compile("\\G\\s*([ugoa]*)([+=-]+)([rwx]+)([,\\s]*)\\s*");
+  final short umaskMode;
+  
+  public UmaskParser(String modeStr) throws IllegalArgumentException {
+    super(modeStr, umaskSymbolicPattern, chmodOctalPattern);
+
+    umaskMode = (short)combineModes(0, false);
+  }
+
+  public short getUMask() {
+    return umaskMode;
+  }
+}

+ 49 - 1
src/test/core/org/apache/hadoop/fs/permission/TestFsPermission.java

@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.fs.permission;
 
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+
 import junit.framework.TestCase;
 
 import static org.apache.hadoop.fs.permission.FsAction.*;
@@ -91,7 +95,6 @@ public class TestFsPermission extends TestCase {
   }
 
   public void testFsPermission() {
-
       String symbolic = "-rwxrwxrwx";
       StringBuilder b = new StringBuilder("-123456789");
 
@@ -113,4 +116,49 @@ public class TestFsPermission extends TestCase {
       }
     }
 
+  public void testUMaskParser() throws IOException {
+    Configuration conf = new Configuration();
+    
+    // Ensure that we get the right octal values back for all legal values
+    for(FsAction u : FsAction.values()) {
+      for(FsAction g : FsAction.values()) {
+        for(FsAction o : FsAction.values()) {
+          FsPermission f = new FsPermission(u, g, o);
+          String asOctal = String.format("%1$03o", f.toShort());
+          conf.set(FsPermission.UMASK_LABEL, asOctal);
+          FsPermission fromConf = FsPermission.getUMask(conf);
+          assertEquals(f, fromConf);
+        }
+      }
+    }
+  }
+
+  public void TestSymbolicUmasks() {
+    Configuration conf = new Configuration();
+    
+    // Test some symbolic settings       Setting       Octal result
+    String [] symbolic = new String [] { "a+rw",        "666",
+                                         "u=x,g=r,o=w", "142",
+                                         "u=x",         "100" };
+    
+    for(int i = 0; i < symbolic.length; i += 2) {
+      conf.set(FsPermission.UMASK_LABEL, symbolic[i]);
+      short val = Short.valueOf(symbolic[i + 1], 8);
+      assertEquals(val, FsPermission.getUMask(conf).toShort());
+    }
+  }
+
+  public void testBadUmasks() {
+    Configuration conf = new Configuration();
+    
+    for(String b : new String [] {"1777", "22", "99", "foo", ""}) {
+      conf.set(FsPermission.UMASK_LABEL, b); 
+      try {
+        FsPermission.getUMask(conf);
+        fail("Shouldn't have been able to parse bad umask");
+      } catch(IllegalArgumentException iae) {
+        assertEquals(iae.getMessage(), b);
+      }
+    }
+  }
 }