瀏覽代碼

HDFS-6326. WebHdfs ACL compatibility is broken. Contributed by Chris Nauroth.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.4@1594975 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 11 年之前
父節點
當前提交
85ca17b33e

+ 21 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -158,6 +158,17 @@ public class FsPermission implements Writable {
     return (short)s;
   }
 
+  /**
+   * Encodes the object to a short.  Unlike {@link #toShort()}, this method may
+   * return values outside the fixed range 00000 - 01777 if extended features
+   * are encoded into this permission, such as the ACL bit.
+   *
+   * @return short extended short representation of this permission
+   */
+  public short toExtendedShort() {
+    return toShort();
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (obj instanceof FsPermission) {
@@ -273,6 +284,16 @@ public class FsPermission implements Writable {
     return stickyBit;
   }
 
+  /**
+   * Returns true if there is also an ACL (access control list).
+   *
+   * @return boolean true if there is also an ACL (access control list).
+   */
+  public boolean getAclBit() {
+    // File system subclasses that support the ACL bit would override this.
+    return false;
+  }
+
   /** Set the user file creation mask (umask) */
   public static void setUMask(Configuration conf, FsPermission umask) {
     conf.set(UMASK_LABEL, String.format("%1$03o", umask.toShort()));

+ 11 - 19
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java

@@ -75,29 +75,21 @@ class AclCommands extends FsCommand {
 
     @Override
     protected void processPath(PathData item) throws IOException {
-      AclStatus aclStatus = item.fs.getAclStatus(item.path);
       out.println("# file: " + item);
-      out.println("# owner: " + aclStatus.getOwner());
-      out.println("# group: " + aclStatus.getGroup());
-      List<AclEntry> entries = aclStatus.getEntries();
-      if (aclStatus.isStickyBit()) {
-        String stickyFlag = "T";
-        for (AclEntry aclEntry : entries) {
-          if (aclEntry.getType() == AclEntryType.OTHER
-              && aclEntry.getScope() == AclEntryScope.ACCESS
-              && aclEntry.getPermission().implies(FsAction.EXECUTE)) {
-            stickyFlag = "t";
-            break;
-          }
-        }
-        out.println("# flags: --" + stickyFlag);
+      out.println("# owner: " + item.stat.getOwner());
+      out.println("# group: " + item.stat.getGroup());
+      FsPermission perm = item.stat.getPermission();
+      if (perm.getStickyBit()) {
+        out.println("# flags: --" +
+          (perm.getOtherAction().implies(FsAction.EXECUTE) ? "t" : "T"));
       }
 
-      FsPermission perm = item.stat.getPermission();
-      if (entries.isEmpty()) {
-        printMinimalAcl(perm);
-      } else {
+      if (perm.getAclBit()) {
+        AclStatus aclStatus = item.fs.getAclStatus(item.path);
+        List<AclEntry> entries = aclStatus.getEntries();
         printExtendedAcl(perm, entries);
+      } else {
+        printMinimalAcl(perm);
       }
 
       out.println();

+ 1 - 46
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java

@@ -31,8 +31,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.RpcNoSuchMethodException;
 
 import com.google.common.collect.Sets;
 
@@ -116,7 +114,7 @@ class Ls extends FsCommand {
     FileStatus stat = item.stat;
     String line = String.format(lineFormat,
         (stat.isDirectory() ? "d" : "-"),
-        stat.getPermission() + (hasAcl(item) ? "+" : " "),
+        stat.getPermission() + (stat.getPermission().getAclBit() ? "+" : " "),
         (stat.isFile() ? stat.getReplication() : "-"),
         stat.getOwner(),
         stat.getGroup(),
@@ -153,49 +151,6 @@ class Ls extends FsCommand {
     lineFormat = fmt.toString();
   }
 
-  /**
-   * Calls getAclStatus to determine if the given item has an ACL.  For
-   * compatibility, this method traps errors caused by the RPC method missing
-   * from the server side.  This would happen if the client was connected to an
-   * old NameNode that didn't have the ACL APIs.  This method also traps the
-   * case of the client-side FileSystem not implementing the ACL APIs.
-   * FileSystem instances that do not support ACLs are remembered.  This
-   * prevents the client from sending multiple failing RPC calls during a
-   * recursive ls.
-   *
-   * @param item PathData item to check
-   * @return boolean true if item has an ACL
-   * @throws IOException if there is a failure
-   */
-  private boolean hasAcl(PathData item) throws IOException {
-    FileSystem fs = item.fs;
-    if (aclNotSupportedFsSet.contains(fs.getUri())) {
-      // This FileSystem failed to run the ACL API in an earlier iteration.
-      return false;
-    }
-    try {
-      return !fs.getAclStatus(item.path).getEntries().isEmpty();
-    } catch (RemoteException e) {
-      // If this is a RpcNoSuchMethodException, then the client is connected to
-      // an older NameNode that doesn't support ACLs.  Keep going.
-      IOException e2 = e.unwrapRemoteException(RpcNoSuchMethodException.class);
-      if (!(e2 instanceof RpcNoSuchMethodException)) {
-        throw e;
-      }
-    } catch (IOException e) {
-      // The NameNode supports ACLs, but they are not enabled.  Keep going.
-      String message = e.getMessage();
-      if (message != null && !message.contains("ACLs has been disabled")) {
-        throw e;
-      }
-    } catch (UnsupportedOperationException e) {
-      // The underlying FileSystem doesn't implement ACLs.  Keep going.
-    }
-    // Remember that this FileSystem cannot support ACLs.
-    aclNotSupportedFsSet.add(fs.getUri());
-    return false;
-  }
-
   private int maxLength(int n, Object value) {
     return Math.max(n, (value != null) ? String.valueOf(value).length() : 0);
   }

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

@@ -66,6 +66,8 @@ Release 2.4.1 - UNRELEASED
     HDFS-6313. WebHdfs may use the wrong NN when configured for multiple HA NNs.
     (kihwal)
 
+    HDFS-6326. WebHdfs ACL compatibility is broken. (cnauroth)
+
 Release 2.4.0 - 2014-04-07 
 
   INCOMPATIBLE CHANGES

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FsAclPermission.java

@@ -0,0 +1,63 @@
+/**
+ * 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.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * HDFS permission subclass used to indicate an ACL is present.  The ACL bit is
+ * not visible directly to users of {@link FsPermission} serialization.  This is
+ * done for backwards compatibility in case any existing clients assume the
+ * value of FsPermission is in a particular range.
+ */
+@InterfaceAudience.Private
+public class FsAclPermission extends FsPermission {
+  private final static short ACL_BIT = 1 << 12;
+  private final boolean aclBit;
+
+  /**
+   * Constructs a new FsAclPermission based on the given FsPermission.
+   *
+   * @param perm FsPermission containing permission bits
+   */
+  public FsAclPermission(FsPermission perm) {
+    super(perm.toShort());
+    aclBit = true;
+  }
+
+  /**
+   * Creates a new FsAclPermission by calling the base class constructor.
+   *
+   * @param perm short containing permission bits
+   */
+  public FsAclPermission(short perm) {
+    super(perm);
+    aclBit = (perm & ACL_BIT) != 0;
+  }
+
+  @Override
+  public short toExtendedShort() {
+    return (short)(toShort() | (aclBit ? ACL_BIT : 0));
+  }
+
+  @Override
+  public boolean getAclBit() {
+    return aclBit;
+  }
+}

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.FsAclPermission;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
@@ -1187,13 +1188,11 @@ public class PBHelper {
   }
   
   public static FsPermissionProto convert(FsPermission p) {
-    if (p == null) return null;
-    return FsPermissionProto.newBuilder().setPerm(p.toShort()).build();
+    return FsPermissionProto.newBuilder().setPerm(p.toExtendedShort()).build();
   }
   
   public static FsPermission convert(FsPermissionProto p) {
-    if (p == null) return null;
-    return new FsPermission((short)p.getPerm());
+    return new FsAclPermission((short)p.getPerm());
   }
   
   

+ 20 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
+import org.apache.hadoop.hdfs.protocol.FsAclPermission;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
@@ -2622,7 +2623,7 @@ public class FSDirectory implements Closeable {
         blocksize,
         node.getModificationTime(snapshot),
         node.getAccessTime(snapshot),
-        node.getFsPermission(snapshot),
+        getPermissionForFileStatus(node, snapshot),
         node.getUserName(snapshot),
         node.getGroupName(snapshot),
         node.isSymlink() ? node.asSymlink().getSymlink() : null,
@@ -2664,7 +2665,8 @@ public class FSDirectory implements Closeable {
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
           blocksize, node.getModificationTime(snapshot),
-          node.getAccessTime(snapshot), node.getFsPermission(snapshot),
+          node.getAccessTime(snapshot),
+          getPermissionForFileStatus(node, snapshot),
           node.getUserName(snapshot), node.getGroupName(snapshot),
           node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
           node.getId(), loc, childrenNum);
@@ -2678,6 +2680,22 @@ public class FSDirectory implements Closeable {
     return status;
   }
 
+  /**
+   * Returns an inode's FsPermission for use in an outbound FileStatus.  If the
+   * inode has an ACL, then this method will convert to a FsAclPermission.
+   *
+   * @param node INode to check
+   * @param snapshot int snapshot ID
+   * @return FsPermission from inode, with ACL bit on if the inode has an ACL
+   */
+  private static FsPermission getPermissionForFileStatus(INode node,
+      int snapshot) {
+    FsPermission perm = node.getFsPermission(snapshot);
+    if (node.getAclFeature(snapshot) != null) {
+      perm = new FsAclPermission(perm);
+    }
+    return perm;
+  }
     
   /**
    * Add the given symbolic link to the fs. Record it in the edits log.

+ 10 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -176,8 +176,9 @@ public class JsonUtil {
   }
 
   /** Convert a string to a FsPermission object. */
-  private static FsPermission toFsPermission(final String s) {
-    return new FsPermission(Short.parseShort(s, 8));
+  private static FsPermission toFsPermission(final String s, Boolean aclBit) {
+    FsPermission perm = new FsPermission(Short.parseShort(s, 8));
+    return (aclBit != null && aclBit) ? new FsAclPermission(perm) : perm;
   }
 
   static enum PathType {
@@ -204,7 +205,11 @@ public class JsonUtil {
     m.put("length", status.getLen());
     m.put("owner", status.getOwner());
     m.put("group", status.getGroup());
-    m.put("permission", toString(status.getPermission()));
+    FsPermission perm = status.getPermission();
+    m.put("permission", toString(perm));
+    if (perm.getAclBit()) {
+      m.put("aclBit", true);
+    }
     m.put("accessTime", status.getAccessTime());
     m.put("modificationTime", status.getModificationTime());
     m.put("blockSize", status.getBlockSize());
@@ -230,7 +235,8 @@ public class JsonUtil {
     final long len = (Long) m.get("length");
     final String owner = (String) m.get("owner");
     final String group = (String) m.get("group");
-    final FsPermission permission = toFsPermission((String) m.get("permission"));
+    final FsPermission permission = toFsPermission((String) m.get("permission"),
+      (Boolean)m.get("aclBit"));
     final long aTime = (Long) m.get("accessTime");
     final long mTime = (Long) m.get("modificationTime");
     final long blockSize = (Long) m.get("blockSize");

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html

@@ -90,7 +90,7 @@
         <tbody>
           {#FileStatus}
           <tr>
-            <td>{#helper_to_permission/}</td>
+            <td>{#helper_to_permission/}{#helper_to_acl_bit/}</td>
             <td>{owner}</td>
             <td>{group}</td>
             <td>{length|fmt_bytes}</td>

+ 10 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js

@@ -26,7 +26,8 @@
       var p = ctx.current().permission;
       var dir = ctx.current().type == 'DIRECTORY' ? 'd' : '-';
       var symbols = [ '---', '--x', '-w-', '-wx', 'r--', 'r-x', 'rw-', 'rwx' ];
-      var sticky = p > 1000;
+      var vInt = parseInt(p, 8);
+      var sticky = (vInt & (1 << 9)) != 0;
 
       var res = "";
       for (var i = 0; i < 3; ++i) {
@@ -35,12 +36,19 @@
       }
 
       if (sticky) {
-        var otherExec = ((ctx.current().permission % 10) & 1) == 1;
+        var otherExec = (vInt & 1) == 1;
         res = res.substr(0, res.length - 1) + (otherExec ? 't' : 'T');
       }
 
       chunk.write(dir + res);
       return chunk;
+    },
+
+    'helper_to_acl_bit': function(chunk, ctx, bodies, params) {
+      if (ctx.current().aclBit) {
+        chunk.write('+');
+      }
+      return chunk;
     }
   };
 

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/AclTestHelpers.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -150,6 +151,9 @@ public final class AclTestHelpers {
    */
   public static void assertPermission(FileSystem fs, Path pathToCheck,
       short perm) throws IOException {
-    assertEquals(perm, fs.getFileStatus(pathToCheck).getPermission().toShort());
+    short filteredPerm = (short)(perm & 01777);
+    FsPermission fsPermission = fs.getFileStatus(pathToCheck).getPermission();
+    assertEquals(filteredPerm, fsPermission.toShort());
+    assertEquals(((perm & (1 << 12)) != 0), fsPermission.getAclBit());
   }
 }

+ 52 - 34
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.AclException;
+import org.apache.hadoop.hdfs.protocol.FsAclPermission;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -118,7 +119,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -140,7 +141,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -161,7 +162,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -177,7 +178,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", READ_WRITE),
       aclEntry(ACCESS, GROUP, READ) }, returned);
-    assertPermission((short)0660);
+    assertPermission((short)010660);
     assertAclFeature(true);
   }
 
@@ -195,7 +196,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, USER, ALL),
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -212,7 +213,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ) }, returned);
-    assertPermission((short)0600);
+    assertPermission((short)010600);
     assertAclFeature(true);
   }
 
@@ -240,7 +241,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)01750);
+    assertPermission((short)011750);
     assertAclFeature(true);
   }
 
@@ -286,7 +287,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -309,7 +310,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bar", READ_WRITE),
       aclEntry(ACCESS, GROUP, READ_WRITE) }, returned);
-    assertPermission((short)0760);
+    assertPermission((short)010760);
     assertAclFeature(true);
   }
 
@@ -334,7 +335,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -382,7 +383,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, USER, ALL),
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -408,7 +409,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)01750);
+    assertPermission((short)011750);
     assertAclFeature(true);
   }
 
@@ -436,7 +437,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission((short)0770);
+    assertPermission((short)010770);
     assertAclFeature(true);
   }
 
@@ -456,7 +457,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission((short)0770);
+    assertPermission((short)010770);
     assertAclFeature(true);
   }
 
@@ -501,7 +502,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission((short)01770);
+    assertPermission((short)011770);
     assertAclFeature(true);
   }
 
@@ -602,7 +603,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0770);
+    assertPermission((short)010770);
     assertAclFeature(true);
   }
 
@@ -621,7 +622,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", READ),
       aclEntry(ACCESS, GROUP, READ) }, returned);
-    assertPermission((short)0640);
+    assertPermission((short)010640);
     assertAclFeature(true);
   }
 
@@ -639,7 +640,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -679,7 +680,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, USER, ALL),
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -699,7 +700,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", READ),
       aclEntry(ACCESS, GROUP, READ) }, returned);
-    assertPermission((short)0670);
+    assertPermission((short)010670);
     assertAclFeature(true);
   }
 
@@ -723,7 +724,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)01770);
+    assertPermission((short)011770);
     assertAclFeature(true);
   }
 
@@ -768,7 +769,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0700);
+    assertPermission((short)010700);
     assertAclFeature(true);
   }
 
@@ -788,7 +789,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", READ),
       aclEntry(ACCESS, GROUP, READ) }, returned);
-    assertPermission((short)0600);
+    assertPermission((short)010600);
     assertAclFeature(true);
   }
 
@@ -810,10 +811,27 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0700);
+    assertPermission((short)010700);
     assertAclFeature(true);
   }
 
+  @Test
+  public void testSetPermissionCannotSetAclBit() throws IOException {
+    FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
+    fs.setPermission(path, FsPermission.createImmutable((short)0700));
+    assertPermission((short)0700);
+    fs.setPermission(path,
+      new FsAclPermission(FsPermission.createImmutable((short)0755)));
+    INode inode = cluster.getNamesystem().getFSDirectory().getRoot().getNode(
+      path.toUri().getPath(), false);
+    assertNotNull(inode);
+    FsPermission perm = inode.getFsPermission();
+    assertNotNull(perm);
+    assertEquals(0755, perm.toShort());
+    assertEquals(0755, perm.toExtendedShort());
+    assertAclFeature(false);
+  }
+
   @Test
   public void testDefaultAclNewFile() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
@@ -827,7 +845,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission(filePath, (short)0640);
+    assertPermission(filePath, (short)010640);
     assertAclFeature(filePath, true);
   }
 
@@ -881,7 +899,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission(dirPath, (short)0750);
+    assertPermission(dirPath, (short)010750);
     assertAclFeature(dirPath, true);
   }
 
@@ -916,7 +934,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, USER, ALL),
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission(dirPath, (short)0750);
+    assertPermission(dirPath, (short)010750);
     assertAclFeature(dirPath, true);
   }
 
@@ -940,7 +958,7 @@ public abstract class FSAclBaseTest {
     AclStatus s = fs.getAclStatus(dirPath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(dirPath, (short)0750);
+    assertPermission(dirPath, (short)010750);
     assertAclFeature(dirPath, true);
     expected = new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
@@ -948,7 +966,7 @@ public abstract class FSAclBaseTest {
     s = fs.getAclStatus(filePath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(filePath, (short)0640);
+    assertPermission(filePath, (short)010640);
     assertAclFeature(filePath, true);
   }
 
@@ -972,12 +990,12 @@ public abstract class FSAclBaseTest {
     AclStatus s = fs.getAclStatus(dirPath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(dirPath, (short)0750);
+    assertPermission(dirPath, (short)010750);
     assertAclFeature(dirPath, true);
     s = fs.getAclStatus(subdirPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(subdirPath, (short)0750);
+    assertPermission(subdirPath, (short)010750);
     assertAclFeature(subdirPath, true);
   }
 
@@ -1004,7 +1022,7 @@ public abstract class FSAclBaseTest {
     AclStatus s = fs.getAclStatus(dirPath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(dirPath, (short)0750);
+    assertPermission(dirPath, (short)010750);
     assertAclFeature(dirPath, true);
     expected = new AclEntry[] { };
     s = fs.getAclStatus(linkPath);
@@ -1037,7 +1055,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission(filePath, (short)0740);
+    assertPermission(filePath, (short)010740);
     assertAclFeature(filePath, true);
   }
 
@@ -1059,7 +1077,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, READ_EXECUTE) }, returned);
-    assertPermission(dirPath, (short)0740);
+    assertPermission(dirPath, (short)010740);
     assertAclFeature(dirPath, true);
   }
 

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithAcl.java

@@ -142,7 +142,7 @@ public class TestFSImageWithAcl {
     AclEntry[] subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
 
     restart(fs, persistNamespace);
 
@@ -152,7 +152,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
 
     aclSpec = Lists.newArrayList(aclEntry(DEFAULT, USER, "foo", READ_WRITE));
     fs.modifyAclEntries(dirPath, aclSpec);
@@ -163,7 +163,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
 
     restart(fs, persistNamespace);
 
@@ -173,7 +173,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
 
     fs.removeAcl(dirPath);
 
@@ -183,7 +183,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
 
     restart(fs, persistNamespace);
 
@@ -193,7 +193,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
   }
 
   @Test

+ 28 - 28
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestAclWithSnapshot.java

@@ -119,14 +119,14 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, path);
+    assertPermission((short)010750, path);
 
     s = hdfs.getAclStatus(snapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, snapshotPath);
+    assertPermission((short)010750, snapshotPath);
 
     assertDirPermissionGranted(fsAsBruce, BRUCE, snapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, snapshotPath);
@@ -153,14 +153,14 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "diana", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0550, path);
+    assertPermission((short)010550, path);
 
     s = hdfs.getAclStatus(snapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, snapshotPath);
+    assertPermission((short)010750, snapshotPath);
 
     assertDirPermissionDenied(fsAsBruce, BRUCE, path);
     assertDirPermissionGranted(fsAsDiana, DIANA, path);
@@ -202,24 +202,24 @@ public class TestAclWithSnapshot {
     AclStatus s = hdfs.getAclStatus(filePath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, filePath);
+    assertPermission((short)010550, filePath);
 
     s = hdfs.getAclStatus(subdirPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirPath);
+    assertPermission((short)010550, subdirPath);
 
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, fileSnapshotPath);
+    assertPermission((short)010550, fileSnapshotPath);
     assertFilePermissionGranted(fsAsBruce, BRUCE, fileSnapshotPath);
     assertFilePermissionDenied(fsAsDiana, DIANA, fileSnapshotPath);
 
     s = hdfs.getAclStatus(subdirSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirSnapshotPath);
+    assertPermission((short)010550, subdirSnapshotPath);
     assertDirPermissionGranted(fsAsBruce, BRUCE, subdirSnapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, subdirSnapshotPath);
 
@@ -251,14 +251,14 @@ public class TestAclWithSnapshot {
     AclStatus s = hdfs.getAclStatus(filePath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0570, filePath);
+    assertPermission((short)010570, filePath);
     assertFilePermissionDenied(fsAsBruce, BRUCE, filePath);
     assertFilePermissionGranted(fsAsDiana, DIANA, filePath);
 
     s = hdfs.getAclStatus(subdirPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0570, subdirPath);
+    assertPermission((short)010570, subdirPath);
     assertDirPermissionDenied(fsAsBruce, BRUCE, subdirPath);
     assertDirPermissionGranted(fsAsDiana, DIANA, subdirPath);
 
@@ -268,14 +268,14 @@ public class TestAclWithSnapshot {
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, fileSnapshotPath);
+    assertPermission((short)010550, fileSnapshotPath);
     assertFilePermissionGranted(fsAsBruce, BRUCE, fileSnapshotPath);
     assertFilePermissionDenied(fsAsDiana, DIANA, fileSnapshotPath);
 
     s = hdfs.getAclStatus(subdirSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirSnapshotPath);
+    assertPermission((short)010550, subdirSnapshotPath);
     assertDirPermissionGranted(fsAsBruce, BRUCE, subdirSnapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, subdirSnapshotPath);
   }
@@ -302,14 +302,14 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, path);
+    assertPermission((short)010750, path);
 
     s = hdfs.getAclStatus(snapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, snapshotPath);
+    assertPermission((short)010750, snapshotPath);
 
     assertDirPermissionGranted(fsAsBruce, BRUCE, snapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, snapshotPath);
@@ -336,7 +336,7 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, snapshotPath);
+    assertPermission((short)010750, snapshotPath);
 
     assertDirPermissionDenied(fsAsBruce, BRUCE, path);
     assertDirPermissionDenied(fsAsDiana, DIANA, path);
@@ -378,24 +378,24 @@ public class TestAclWithSnapshot {
     AclStatus s = hdfs.getAclStatus(filePath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, filePath);
+    assertPermission((short)010550, filePath);
 
     s = hdfs.getAclStatus(subdirPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirPath);
+    assertPermission((short)010550, subdirPath);
 
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, fileSnapshotPath);
+    assertPermission((short)010550, fileSnapshotPath);
     assertFilePermissionGranted(fsAsBruce, BRUCE, fileSnapshotPath);
     assertFilePermissionDenied(fsAsDiana, DIANA, fileSnapshotPath);
 
     s = hdfs.getAclStatus(subdirSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirSnapshotPath);
+    assertPermission((short)010550, subdirSnapshotPath);
     assertDirPermissionGranted(fsAsBruce, BRUCE, subdirSnapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, subdirSnapshotPath);
 
@@ -437,14 +437,14 @@ public class TestAclWithSnapshot {
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, fileSnapshotPath);
+    assertPermission((short)010550, fileSnapshotPath);
     assertFilePermissionGranted(fsAsBruce, BRUCE, fileSnapshotPath);
     assertFilePermissionDenied(fsAsDiana, DIANA, fileSnapshotPath);
 
     s = hdfs.getAclStatus(subdirSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirSnapshotPath);
+    assertPermission((short)010550, subdirSnapshotPath);
     assertDirPermissionGranted(fsAsBruce, BRUCE, subdirSnapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, subdirSnapshotPath);
   }
@@ -470,7 +470,7 @@ public class TestAclWithSnapshot {
     AclStatus s = hdfs.getAclStatus(path);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0770, path);
+    assertPermission((short)010770, path);
     assertDirPermissionGranted(fsAsBruce, BRUCE, path);
     assertDirPermissionGranted(fsAsDiana, DIANA, path);
   }
@@ -514,7 +514,7 @@ public class TestAclWithSnapshot {
       aclEntry(DEFAULT, GROUP, NONE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0700, path);
+    assertPermission((short)010700, path);
 
     s = hdfs.getAclStatus(snapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
@@ -524,7 +524,7 @@ public class TestAclWithSnapshot {
       aclEntry(DEFAULT, GROUP, NONE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0700, snapshotPath);
+    assertPermission((short)010700, snapshotPath);
 
     assertDirPermissionDenied(fsAsBruce, BRUCE, snapshotPath);
   }
@@ -596,14 +596,14 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_WRITE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0660, filePath);
+    assertPermission((short)010660, filePath);
 
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_WRITE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0660, filePath);
+    assertPermission((short)010660, filePath);
 
     aclSpec = Lists.newArrayList(
       aclEntry(ACCESS, USER, "bruce", READ));
@@ -632,14 +632,14 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_WRITE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0660, filePath);
+    assertPermission((short)010660, filePath);
 
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_WRITE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0660, filePath);
+    assertPermission((short)010660, filePath);
 
     aclSpec = Lists.newArrayList(
       aclEntry(ACCESS, USER, "bruce", READ));