Browse Source

HDFS-6962. ACL inheritance conflicts with umaskmode. Contributed by Chris Nauroth.

Chris Nauroth 8 năm trước cách đây
mục cha
commit
f0d5382ff3
29 tập tin đã thay đổi với 1613 bổ sung54 xóa
  1. 5 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
  2. 4 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  3. 101 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsCreateModes.java
  4. 14 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
  5. 25 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/package-info.java
  6. 3 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  7. 11 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  8. 12 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  9. 7 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PermissionParam.java
  10. 51 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/UnmaskedPermissionParam.java
  11. 2 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
  12. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  13. 13 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  14. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/ParameterParser.java
  15. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java
  16. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
  17. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
  18. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSymlinkOp.java
  19. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
  20. 67 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  21. 32 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  22. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  23. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsPermissionsGuide.md
  24. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestAclCLI.java
  25. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestAclCLIWithPosixAclInheritance.java
  26. 91 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
  27. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
  28. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  29. 1075 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testAclCLIWithPosixAclInheritance.xml

+ 5 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.Options.CreateOpts;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsCreateModes;
 import org.apache.hadoop.fs.permission.FsPermission;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_DEFAULT;
@@ -671,7 +672,7 @@ public class FileContext {
     CreateOpts.Perms permOpt = CreateOpts.getOpt(CreateOpts.Perms.class, opts);
     FsPermission permission = (permOpt != null) ? permOpt.getValue() :
                                       FILE_DEFAULT_PERM;
-    permission = permission.applyUMask(getUMask());
+    permission = FsCreateModes.applyUMask(permission, getUMask());
 
     final CreateOpts[] updatedOpts = 
                       CreateOpts.setOpt(CreateOpts.perms(permission), opts);
@@ -717,8 +718,9 @@ public class FileContext {
       ParentNotDirectoryException, UnsupportedFileSystemException, 
       IOException {
     final Path absDir = fixRelativePart(dir);
-    final FsPermission absFerms = (permission == null ? 
-          FsPermission.getDirDefault() : permission).applyUMask(getUMask());
+    final FsPermission absFerms = FsCreateModes.applyUMask(
+        permission == null ?
+            FsPermission.getDirDefault() : permission, getUMask());
     new FSLinkResolver<Void>() {
       @Override
       public Void next(final AbstractFileSystem fs, final Path p) 

+ 4 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -54,6 +54,7 @@ import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsCreateModes;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.io.Text;
@@ -925,9 +926,9 @@ public abstract class FileSystem extends Configured implements Closeable {
                                             long blockSize,
                                             Progressable progress
                                             ) throws IOException {
-    return this.create(f, FsPermission.getFileDefault().applyUMask(
-        FsPermission.getUMask(getConf())), overwrite, bufferSize,
-        replication, blockSize, progress);
+    return this.create(f, FsCreateModes.applyUMask(
+        FsPermission.getFileDefault(), FsPermission.getUMask(getConf())),
+        overwrite, bufferSize, replication, blockSize, progress);
   }
 
   /**

+ 101 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsCreateModes.java

@@ -0,0 +1,101 @@
+/**
+ * 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.text.MessageFormat;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class that stores both masked and unmasked create modes
+ * and is a drop-in replacement for masked permission.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class FsCreateModes extends FsPermission {
+  private final FsPermission unmasked;
+
+  /**
+   * Create from unmasked mode and umask.
+   *
+   * If the mode is already an FsCreateModes object, return it.
+   */
+  public static FsPermission applyUMask(FsPermission mode,
+                                        FsPermission umask) {
+    if (mode.getUnmasked() != null) {
+      return mode;
+    }
+    return create(mode.applyUMask(umask), mode);
+  }
+
+  /**
+   * Create from masked and unmasked modes.
+   */
+  public static FsCreateModes create(FsPermission masked,
+                                     FsPermission unmasked) {
+    assert masked.getUnmasked() == null;
+    assert unmasked.getUnmasked() == null;
+    return new FsCreateModes(masked, unmasked);
+  }
+
+  private FsCreateModes(FsPermission masked, FsPermission unmasked) {
+    super(masked);
+    this.unmasked = unmasked;
+    assert masked.getUnmasked() == null;
+    assert unmasked.getUnmasked() == null;
+  }
+
+  @Override
+  public FsPermission getMasked() {
+    return this;
+  }
+
+  @Override
+  public FsPermission getUnmasked() {
+    return unmasked;
+  }
+
+  @Override
+  public String toString() {
+    return MessageFormat.format("'{' masked: {0}, unmasked: {1} '}'",
+        super.toString(), getUnmasked());
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    FsCreateModes that = (FsCreateModes) o;
+    return getUnmasked().equals(that.getUnmasked());
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + getUnmasked().hashCode();
+    return result;
+  }
+}

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

@@ -137,6 +137,20 @@ public class FsPermission implements Writable {
     fromShort(in.readShort());
   }
 
+  /**
+   * Get masked permission if exists.
+   */
+  public FsPermission getMasked() {
+    return null;
+  }
+
+  /**
+   * Get unmasked permission if exists.
+   */
+  public FsPermission getUnmasked() {
+    return null;
+  }
+
   /**
    * Create and initialize a {@link FsPermission} from {@link DataInput}.
    */

+ 25 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/package-info.java

@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+/**
+ * This package provides support for HDFS permission and ACL.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.permission;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -90,6 +90,7 @@ import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsCreateModes;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -1160,14 +1161,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if (permission == null) {
       permission = FsPermission.getFileDefault();
     }
-    return permission.applyUMask(dfsClientConf.getUMask());
+    return FsCreateModes.applyUMask(permission, dfsClientConf.getUMask());
   }
 
   private FsPermission applyUMaskDir(FsPermission permission) {
     if (permission == null) {
       permission = FsPermission.getDirDefault();
     }
-    return permission.applyUMask(dfsClientConf.getUMask());
+    return FsCreateModes.applyUMask(permission, dfsClientConf.getUMask());
   }
 
   /**

+ 11 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -294,6 +294,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setCreateParent(createParent)
         .setReplication(replication)
         .setBlockSize(blockSize);
+    FsPermission unmasked = masked.getUnmasked();
+    if (unmasked != null) {
+      builder.setUnmasked(PBHelperClient.convert(unmasked));
+    }
     builder.addAllCryptoProtocolVersion(
         PBHelperClient.convert(supportedVersions));
     CreateRequestProto req = builder.build();
@@ -579,11 +583,15 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
       throws IOException {
-    MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
+    MkdirsRequestProto.Builder builder = MkdirsRequestProto.newBuilder()
         .setSrc(src)
         .setMasked(PBHelperClient.convert(masked))
-        .setCreateParent(createParent).build();
-
+        .setCreateParent(createParent);
+    FsPermission unmasked = masked.getUnmasked();
+    if (unmasked != null) {
+      builder.setUnmasked(PBHelperClient.convert(unmasked));
+    }
+    MkdirsRequestProto req = builder.build();
     try {
       return rpcProxy.mkdirs(null, req).getResult();
     } catch (ServiceException e) {

+ 12 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageStatistics;
+import org.apache.hadoop.fs.permission.FsCreateModes;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
@@ -973,7 +974,8 @@ public class WebHdfsFileSystem extends FileSystem
     if (permission == null) {
       permission = FsPermission.getDefault();
     }
-    return permission.applyUMask(FsPermission.getUMask(getConf()));
+    return FsCreateModes.applyUMask(permission,
+        FsPermission.getUMask(getConf()));
   }
 
   private HdfsFileStatus getHdfsFileStatus(Path f) throws IOException {
@@ -1025,8 +1027,10 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementWriteOps(1);
     storageStatistics.incrementOpCounter(OpType.MKDIRS);
     final HttpOpParam.Op op = PutOpParam.Op.MKDIRS;
+    final FsPermission modes = applyUMask(permission);
     return new FsPathBooleanRunner(op, f,
-        new PermissionParam(applyUMask(permission))
+        new PermissionParam(modes.getMasked()),
+        new UnmaskedPermissionParam(modes.getUnmasked())
     ).run();
   }
 
@@ -1313,9 +1317,11 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementWriteOps(1);
     storageStatistics.incrementOpCounter(OpType.CREATE);
 
+    final FsPermission modes = applyUMask(permission);
     final HttpOpParam.Op op = PutOpParam.Op.CREATE;
     return new FsPathOutputStreamRunner(op, f, bufferSize,
-        new PermissionParam(applyUMask(permission)),
+        new PermissionParam(modes.getMasked()),
+        new UnmaskedPermissionParam(modes.getUnmasked()),
         new OverwriteParam(overwrite),
         new BufferSizeParam(bufferSize),
         new ReplicationParam(replication),
@@ -1331,9 +1337,11 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementWriteOps(1);
     storageStatistics.incrementOpCounter(OpType.CREATE_NON_RECURSIVE);
 
+    final FsPermission modes = applyUMask(permission);
     final HttpOpParam.Op op = PutOpParam.Op.CREATE;
     return new FsPathOutputStreamRunner(op, f, bufferSize,
-        new PermissionParam(applyUMask(permission)),
+        new PermissionParam(modes.getMasked()),
+        new UnmaskedPermissionParam(modes.getUnmasked()),
         new CreateFlagParam(flag),
         new CreateParentParam(false),
         new BufferSizeParam(bufferSize),

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PermissionParam.java

@@ -54,7 +54,7 @@ public class PermissionParam extends ShortParam {
    * @param value the parameter value.
    */
   public PermissionParam(final FsPermission value) {
-    super(DOMAIN, value == null? null: value.toShort(), null, null);
+    this(DOMAIN, value == null ? null : value.toShort(), null, null);
   }
 
   /**
@@ -62,7 +62,12 @@ public class PermissionParam extends ShortParam {
    * @param str a string representation of the parameter value.
    */
   public PermissionParam(final String str) {
-    super(DOMAIN, DOMAIN.parse(str), (short)0, (short)01777);
+    this(DOMAIN, DOMAIN.parse(str), (short)0, (short)01777);
+  }
+
+  PermissionParam(final Domain domain, final Short value, final Short min,
+                  final Short max) {
+    super(domain, value, min, max);
   }
 
   @Override

+ 51 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/UnmaskedPermissionParam.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.hdfs.web.resources;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Unmasked permission parameter, use a Short to represent a FsPermission.
+ */
+public class UnmaskedPermissionParam extends PermissionParam {
+  /** Parameter name. */
+  public static final String NAME = "unmaskedpermission";
+
+  private static final Domain DOMAIN = new Domain(NAME, 8);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public UnmaskedPermissionParam(final FsPermission value) {
+    super(DOMAIN, value == null ? null : value.toShort(), null, null);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public UnmaskedPermissionParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str), (short)0, (short)01777);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto

@@ -79,6 +79,7 @@ message CreateRequestProto {
   required uint32 replication = 6; // Short: Only 16 bits used
   required uint64 blockSize = 7;
   repeated CryptoProtocolVersionProto cryptoProtocolVersion = 8;
+  optional FsPermissionProto unmasked = 9;
 }
 
 message CreateResponseProto {
@@ -264,6 +265,7 @@ message MkdirsRequestProto {
   required string src = 1;
   required FsPermissionProto masked = 2;
   required bool createParent = 3;
+  optional FsPermissionProto unmasked = 4;
 }
 message MkdirsResponseProto {
     required bool result = 1;

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -259,6 +259,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT = "supergroup";
   public static final String  DFS_NAMENODE_ACLS_ENABLED_KEY = "dfs.namenode.acls.enabled";
   public static final boolean DFS_NAMENODE_ACLS_ENABLED_DEFAULT = false;
+  public static final String DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_KEY =
+      "dfs.namenode.posix.acl.inheritance.enabled";
+  public static final boolean
+      DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_DEFAULT = false;
   public static final String  DFS_NAMENODE_XATTRS_ENABLED_KEY = "dfs.namenode.xattrs.enabled";
   public static final boolean DFS_NAMENODE_XATTRS_ENABLED_DEFAULT = true;
   public static final String  DFS_ADMIN = "dfs.cluster.administrators";

+ 13 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -25,6 +25,8 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.permission.FsCreateModes;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.AddBlockFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -411,8 +413,12 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public CreateResponseProto create(RpcController controller,
       CreateRequestProto req) throws ServiceException {
     try {
+      FsPermission masked = req.hasUnmasked() ?
+          FsCreateModes.create(PBHelperClient.convert(req.getMasked()),
+              PBHelperClient.convert(req.getUnmasked())) :
+          PBHelperClient.convert(req.getMasked());
       HdfsFileStatus result = server.create(req.getSrc(),
-          PBHelperClient.convert(req.getMasked()), req.getClientName(),
+          masked, req.getClientName(),
           PBHelperClient.convertCreateFlag(req.getCreateFlag()), req.getCreateParent(),
           (short) req.getReplication(), req.getBlockSize(),
           PBHelperClient.convertCryptoProtocolVersions(
@@ -651,8 +657,12 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public MkdirsResponseProto mkdirs(RpcController controller,
       MkdirsRequestProto req) throws ServiceException {
     try {
-      boolean result = server.mkdirs(req.getSrc(),
-          PBHelperClient.convert(req.getMasked()), req.getCreateParent());
+      FsPermission masked = req.hasUnmasked() ?
+          FsCreateModes.create(PBHelperClient.convert(req.getMasked()),
+              PBHelperClient.convert(req.getUnmasked())) :
+          PBHelperClient.convert(req.getMasked());
+      boolean result = server.mkdirs(req.getSrc(), masked,
+          req.getCreateParent());
       return MkdirsResponseProto.newBuilder().setResult(result).build();
     } catch (IOException e) {
       throw new ServiceException(e);

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/ParameterParser.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.web.resources.OffsetParam;
 import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
 import org.apache.hadoop.hdfs.web.resources.PermissionParam;
 import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
+import org.apache.hadoop.hdfs.web.resources.UnmaskedPermissionParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.token.Token;
@@ -108,6 +109,12 @@ class ParameterParser {
         getFileFsPermission();
   }
 
+  FsPermission unmaskedPermission() {
+    String value = param(UnmaskedPermissionParam.NAME);
+    return value == null ? null :
+        new UnmaskedPermissionParam(value).getFileFsPermission();
+  }
+
   boolean overwrite() {
     return new OverwriteParam(param(OverwriteParam.NAME)).getValue();
   }

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java

@@ -66,6 +66,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.fs.permission.FsCreateModes;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
@@ -187,7 +188,10 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
     final int bufferSize = params.bufferSize();
     final short replication = params.replication();
     final long blockSize = params.blockSize();
-    final FsPermission permission = params.permission();
+    final FsPermission unmaskedPermission = params.unmaskedPermission();
+    final FsPermission permission = unmaskedPermission == null ?
+        params.permission() :
+        FsCreateModes.create(params.permission(), unmaskedPermission);
     final boolean createParent = params.createParent();
 
     EnumSet<CreateFlag> flags = params.createFlag();

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java

@@ -73,11 +73,11 @@ public final class AclStorage {
    *
    * @param child INode newly created child
    */
-  public static void copyINodeDefaultAcl(INode child) {
+  public static boolean copyINodeDefaultAcl(INode child) {
     INodeDirectory parent = child.getParent();
     AclFeature parentAclFeature = parent.getAclFeature();
     if (parentAclFeature == null || !(child.isFile() || child.isDirectory())) {
-      return;
+      return false;
     }
 
     // Split parent's entries into access vs. default.
@@ -88,7 +88,7 @@ public final class AclStorage {
 
     // The parent may have an access ACL but no default ACL.  If so, exit.
     if (parentDefaultEntries.isEmpty()) {
-      return;
+      return false;
     }
 
     // Pre-allocate list size for access entries to copy from parent.
@@ -145,6 +145,7 @@ public final class AclStorage {
     }
 
     child.setPermission(newPerm);
+    return true;
   }
 
   /**

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java

@@ -219,7 +219,8 @@ class FSDirMkdirOp {
     final INodeDirectory dir = new INodeDirectory(inodeId, name, permission,
         timestamp);
 
-    INodesInPath iip = fsd.addLastINode(parent, dir, true);
+    INodesInPath iip =
+        fsd.addLastINode(parent, dir, permission.getPermission(), true);
     if (iip != null && aclEntries != null) {
       AclStorage.updateINodeAcl(dir, aclEntries, Snapshot.CURRENT_STATE_ID);
     }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSymlinkOp.java

@@ -88,7 +88,8 @@ class FSDirSymlinkOp {
     final INodeSymlink symlink = new INodeSymlink(id, null, perm, mtime, atime,
         target);
     symlink.setLocalName(localName);
-    return fsd.addINode(iip, symlink) != null ? symlink : null;
+    return fsd.addINode(iip, symlink, perm.getPermission()) != null ?
+        symlink : null;
   }
 
   /**

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java

@@ -496,7 +496,8 @@ class FSDirWriteFileOp {
             replication, preferredBlockSize, storagePolicyId, ecPolicy != null);
       }
       newNode.setLocalName(localName);
-      INodesInPath iip = fsd.addINode(existing, newNode);
+      INodesInPath iip = fsd.addINode(existing, newNode,
+          permissions.getPermission());
       if (iip != null) {
         if (aclEntries != null) {
           AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID);
@@ -594,7 +595,7 @@ class FSDirWriteFileOp {
           modTime, modTime, replication, preferredBlockSize, ecPolicy != null);
       newNode.setLocalName(localName);
       newNode.toUnderConstruction(clientName, clientMachine);
-      newiip = fsd.addINode(existing, newNode);
+      newiip = fsd.addINode(existing, newNode, permissions.getPermission());
     } finally {
       fsd.writeUnlock();
     }

+ 67 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -168,6 +168,10 @@ public class FSDirectory implements Closeable {
    * ACL-related operations.
    */
   private final boolean aclsEnabled;
+  /**
+   * Support for POSIX ACL inheritance. Not final for testing purpose.
+   */
+  private boolean posixAclInheritanceEnabled;
   private final boolean xattrsEnabled;
   private final int xattrMaxSize;
 
@@ -251,6 +255,10 @@ public class FSDirectory implements Closeable {
         DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,
         DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_DEFAULT);
     LOG.info("ACLs enabled? " + aclsEnabled);
+    this.posixAclInheritanceEnabled = conf.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_KEY,
+        DFSConfigKeys.DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_DEFAULT);
+    LOG.info("POSIX ACL inheritance enabled? " + posixAclInheritanceEnabled);
     this.xattrsEnabled = conf.getBoolean(
         DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY,
         DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_DEFAULT);
@@ -450,6 +458,18 @@ public class FSDirectory implements Closeable {
   boolean isAclsEnabled() {
     return aclsEnabled;
   }
+
+  @VisibleForTesting
+  public boolean isPosixAclInheritanceEnabled() {
+    return posixAclInheritanceEnabled;
+  }
+
+  @VisibleForTesting
+  public void setPosixAclInheritanceEnabled(
+      boolean posixAclInheritanceEnabled) {
+    this.posixAclInheritanceEnabled = posixAclInheritanceEnabled;
+  }
+
   boolean isXattrsEnabled() {
     return xattrsEnabled;
   }
@@ -952,16 +972,18 @@ public class FSDirectory implements Closeable {
    * Add the given child to the namespace.
    * @param existing the INodesInPath containing all the ancestral INodes
    * @param child the new INode to add
+   * @param modes create modes
    * @return a new INodesInPath instance containing the new child INode. Null
    * if the adding fails.
    * @throws QuotaExceededException is thrown if it violates quota limit
    */
-  INodesInPath addINode(INodesInPath existing, INode child)
+  INodesInPath addINode(INodesInPath existing, INode child,
+                        FsPermission modes)
       throws QuotaExceededException, UnresolvedLinkException {
     cacheName(child);
     writeLock();
     try {
-      return addLastINode(existing, child, true);
+      return addLastINode(existing, child, modes, true);
     } finally {
       writeUnlock();
     }
@@ -1066,13 +1088,52 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  /**
+   * Turn on HDFS-6962 POSIX ACL inheritance when the property
+   * {@link DFSConfigKeys#DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_KEY} is
+   * true and a compatible client has sent both masked and unmasked create
+   * modes.
+   *
+   * @param child INode newly created child
+   * @param modes create modes
+   */
+  private void copyINodeDefaultAcl(INode child, FsPermission modes) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("child: {}, posixAclInheritanceEnabled: {}, modes: {}",
+          child, posixAclInheritanceEnabled, modes);
+    }
+
+    if (posixAclInheritanceEnabled && modes != null &&
+        modes.getUnmasked() != null) {
+      //
+      // HDFS-6962: POSIX ACL inheritance
+      //
+      child.setPermission(modes.getUnmasked());
+      if (!AclStorage.copyINodeDefaultAcl(child)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("{}: no parent default ACL to inherit", child);
+        }
+        child.setPermission(modes.getMasked());
+      }
+    } else {
+      //
+      // Old behavior before HDFS-6962
+      //
+      AclStorage.copyINodeDefaultAcl(child);
+    }
+  }
+
   /**
    * Add a child to the end of the path specified by INodesInPath.
+   * @param existing the INodesInPath containing all the ancestral INodes
+   * @param inode the new INode to add
+   * @param modes create modes
+   * @param checkQuota whether to check quota
    * @return an INodesInPath instance containing the new INode
    */
   @VisibleForTesting
   public INodesInPath addLastINode(INodesInPath existing, INode inode,
-      boolean checkQuota) throws QuotaExceededException {
+      FsPermission modes, boolean checkQuota) throws QuotaExceededException {
     assert existing.getLastINode() != null &&
         existing.getLastINode().isDirectory();
 
@@ -1119,7 +1180,7 @@ public class FSDirectory implements Closeable {
       return null;
     } else {
       if (!isRename) {
-        AclStorage.copyINodeDefaultAcl(inode);
+        copyINodeDefaultAcl(inode, modes);
       }
       addToInodeMap(inode);
     }
@@ -1128,7 +1189,8 @@ public class FSDirectory implements Closeable {
 
   INodesInPath addLastINodeNoQuotaCheck(INodesInPath existing, INode i) {
     try {
-      return addLastINode(existing, i, false);
+      // All callers do not have create modes to pass.
+      return addLastINode(existing, i, null, false);
     } catch (QuotaExceededException e) {
       NameNode.LOG.warn("FSDirectory.addChildNoQuotaCheck - unexpected", e);
     }

+ 32 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -58,6 +58,8 @@ import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsCreateModes;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -324,6 +326,9 @@ public class NamenodeWebHdfsMethods {
           final GroupParam group,
       @QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT)
           final PermissionParam permission,
+      @QueryParam(UnmaskedPermissionParam.NAME)
+      @DefaultValue(UnmaskedPermissionParam.DEFAULT)
+          final UnmaskedPermissionParam unmaskedPermission,
       @QueryParam(OverwriteParam.NAME) @DefaultValue(OverwriteParam.DEFAULT)
           final OverwriteParam overwrite,
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
@@ -362,11 +367,11 @@ public class NamenodeWebHdfsMethods {
           final NoRedirectParam noredirect
       ) throws IOException, InterruptedException {
     return put(ugi, delegation, username, doAsUser, ROOT, op, destination,
-        owner, group, permission, overwrite, bufferSize, replication,
-        blockSize, modificationTime, accessTime, renameOptions, createParent,
-        delegationTokenArgument, aclPermission, xattrName, xattrValue,
-        xattrSetFlag, snapshotName, oldSnapshotName, excludeDatanodes,
-        createFlagParam, noredirect);
+        owner, group, permission, unmaskedPermission, overwrite, bufferSize,
+        replication, blockSize, modificationTime, accessTime, renameOptions,
+        createParent, delegationTokenArgument, aclPermission, xattrName,
+        xattrValue, xattrSetFlag, snapshotName, oldSnapshotName,
+        excludeDatanodes, createFlagParam, noredirect);
   }
 
   /** Handle HTTP PUT request. */
@@ -393,6 +398,9 @@ public class NamenodeWebHdfsMethods {
           final GroupParam group,
       @QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT)
           final PermissionParam permission,
+      @QueryParam(UnmaskedPermissionParam.NAME)
+      @DefaultValue(UnmaskedPermissionParam.DEFAULT)
+          final UnmaskedPermissionParam unmaskedPermission,
       @QueryParam(OverwriteParam.NAME) @DefaultValue(OverwriteParam.DEFAULT)
           final OverwriteParam overwrite,
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
@@ -432,10 +440,11 @@ public class NamenodeWebHdfsMethods {
       ) throws IOException, InterruptedException {
 
     init(ugi, delegation, username, doAsUser, path, op, destination, owner,
-        group, permission, overwrite, bufferSize, replication, blockSize,
-        modificationTime, accessTime, renameOptions, delegationTokenArgument,
-        aclPermission, xattrName, xattrValue, xattrSetFlag, snapshotName,
-        oldSnapshotName, excludeDatanodes, createFlagParam, noredirect);
+        group, permission, unmaskedPermission, overwrite, bufferSize,
+        replication, blockSize, modificationTime, accessTime, renameOptions,
+        delegationTokenArgument, aclPermission, xattrName, xattrValue,
+        xattrSetFlag, snapshotName, oldSnapshotName, excludeDatanodes,
+        createFlagParam, noredirect);
 
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
@@ -443,10 +452,11 @@ public class NamenodeWebHdfsMethods {
         try {
           return put(ugi, delegation, username, doAsUser,
               path.getAbsolutePath(), op, destination, owner, group,
-              permission, overwrite, bufferSize, replication, blockSize,
-              modificationTime, accessTime, renameOptions, createParent,
-              delegationTokenArgument, aclPermission, xattrName, xattrValue,
-              xattrSetFlag, snapshotName, oldSnapshotName, excludeDatanodes,
+              permission, unmaskedPermission, overwrite, bufferSize,
+              replication, blockSize, modificationTime, accessTime,
+              renameOptions, createParent, delegationTokenArgument,
+              aclPermission, xattrName, xattrValue, xattrSetFlag,
+              snapshotName, oldSnapshotName, excludeDatanodes,
               createFlagParam, noredirect);
         } finally {
           reset();
@@ -466,6 +476,7 @@ public class NamenodeWebHdfsMethods {
       final OwnerParam owner,
       final GroupParam group,
       final PermissionParam permission,
+      final UnmaskedPermissionParam unmaskedPermission,
       final OverwriteParam overwrite,
       final BufferSizeParam bufferSize,
       final ReplicationParam replication,
@@ -495,8 +506,9 @@ public class NamenodeWebHdfsMethods {
     {
       final URI uri = redirectURI(namenode, ugi, delegation, username,
           doAsUser, fullpath, op.getValue(), -1L, blockSize.getValue(conf),
-          exclDatanodes.getValue(), permission, overwrite, bufferSize,
-          replication, blockSize, createParent, createFlagParam);
+          exclDatanodes.getValue(), permission, unmaskedPermission,
+          overwrite, bufferSize, replication, blockSize, createParent,
+          createFlagParam);
       if(!noredirectParam.getValue()) {
         return Response.temporaryRedirect(uri)
           .type(MediaType.APPLICATION_OCTET_STREAM).build();
@@ -507,8 +519,11 @@ public class NamenodeWebHdfsMethods {
     }
     case MKDIRS:
     {
-      final boolean b = np.mkdirs(fullpath,
-          permission.getDirFsPermission(), true);
+      FsPermission masked = unmaskedPermission.getValue() == null ?
+          permission.getDirFsPermission() :
+          FsCreateModes.create(permission.getDirFsPermission(),
+              unmaskedPermission.getDirFsPermission());
+      final boolean b = np.mkdirs(fullpath, masked, true);
       final String js = JsonUtil.toJsonString("boolean", b);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -457,7 +457,19 @@
   </description>
 </property>
 
-<property>
+  <property>
+    <name>dfs.namenode.posix.acl.inheritance.enabled</name>
+    <value>false</value>
+    <description>
+      Set to true to enable POSIX style ACL inheritance. When it is enabled
+      and the create request comes from a compatible client, the NameNode
+      will apply default ACLs from the parent directory to the create mode
+      and ignore the client umask. If no default ACL found, it will apply the
+      client umask.
+    </description>
+  </property>
+
+  <property>
   <name>dfs.namenode.lazypersist.file.scrub.interval.sec</name>
   <value>300</value>
   <description>

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsPermissionsGuide.md

@@ -333,4 +333,10 @@ Configuration Parameters
     default, ACLs are disabled. When ACLs are disabled, the NameNode rejects
     all attempts to set an ACL.
 
+*   `dfs.namenode.posix.acl.inheritance.enabled`
 
+    Set to true to enable POSIX style ACL inheritance. Disabled by default.
+    When it is enabled and the create request comes from a compatible client,
+    the NameNode will apply default ACLs from the parent directory to
+    the create mode and ignore the client umask. If no default ACL is found,
+    it will apply the client umask.

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestAclCLI.java

@@ -32,11 +32,15 @@ public class TestAclCLI extends CLITestHelperDFS {
   private String namenode = null;
   private String username = null;
 
+  protected void initConf() {
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+  }
+
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    initConf();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     fs = cluster.getFileSystem();
     namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///");

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestAclCLIWithPosixAclInheritance.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.cli;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_KEY;
+
+import org.junit.Test;
+
+/**
+ * Test ACL CLI with POSIX ACL inheritance enabled.
+ */
+public class TestAclCLIWithPosixAclInheritance extends TestAclCLI {
+
+  @Override
+  protected void initConf() {
+    super.initConf();
+    conf.setBoolean(DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_KEY, true);
+  }
+
+  @Override
+  protected String getTestFile() {
+    return "testAclCLIWithPosixAclInheritance.xml";
+  }
+
+  @Test
+  @Override
+  public void testAll() {
+    super.testAll();
+  }
+}

+ 91 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.*;
 import static org.apache.hadoop.fs.permission.AclEntryScope.*;
 import static org.apache.hadoop.fs.permission.AclEntryType.*;
@@ -888,6 +889,46 @@ public abstract class FSAclBaseTest {
     assertAclFeature(filePath, true);
   }
 
+  @Test
+  public void testUMaskDefaultAclNewFile() throws Exception {
+    FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
+    List<AclEntry> aclSpec = Lists.newArrayList(
+        aclEntry(DEFAULT, GROUP, READ_WRITE),
+        aclEntry(DEFAULT, USER, "foo", ALL));
+    fs.setAcl(path, aclSpec);
+
+    String oldUMask = fs.getConf().get(FS_PERMISSIONS_UMASK_KEY);
+    fs.getConf().set(FS_PERMISSIONS_UMASK_KEY, "027");
+
+    FSDirectory fsDirectory = cluster.getNamesystem().getFSDirectory();
+    boolean oldEnabled = fsDirectory.isPosixAclInheritanceEnabled();
+
+    try {
+      fsDirectory.setPosixAclInheritanceEnabled(false);
+      Path filePath = new Path(path, "file1");
+      fs.create(filePath).close();
+      AclStatus s = fs.getAclStatus(filePath);
+      AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
+      assertArrayEquals(new AclEntry[]{
+          aclEntry(ACCESS, USER, "foo", ALL),
+          aclEntry(ACCESS, GROUP, READ_WRITE)}, returned);
+      assertPermission(filePath, (short) 010640);
+
+      fsDirectory.setPosixAclInheritanceEnabled(true);
+      Path file2Path = new Path(path, "file2");
+      fs.create(file2Path).close();
+      AclStatus s2 = fs.getAclStatus(file2Path);
+      AclEntry[] returned2 = s2.getEntries().toArray(new AclEntry[0]);
+      assertArrayEquals(new AclEntry[]{
+          aclEntry(ACCESS, USER, "foo", ALL),
+          aclEntry(ACCESS, GROUP, READ_WRITE)}, returned2);
+      assertPermission(file2Path, (short) 010660);
+    } finally {
+      fsDirectory.setPosixAclInheritanceEnabled(oldEnabled);
+      fs.getConf().set(FS_PERMISSIONS_UMASK_KEY, oldUMask);
+    }
+  }
+
   @Test
   public void testOnlyAccessAclNewFile() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
@@ -942,6 +983,56 @@ public abstract class FSAclBaseTest {
     assertAclFeature(dirPath, true);
   }
 
+  @Test
+  public void testUMaskDefaultAclNewDir() throws Exception {
+    FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
+    List<AclEntry> aclSpec = Lists.newArrayList(
+        aclEntry(DEFAULT, GROUP, ALL),
+        aclEntry(DEFAULT, USER, "foo", ALL));
+    fs.setAcl(path, aclSpec);
+
+    String oldUMask = fs.getConf().get(FS_PERMISSIONS_UMASK_KEY);
+    fs.getConf().set(FS_PERMISSIONS_UMASK_KEY, "027");
+
+    FSDirectory fsDirectory = cluster.getNamesystem().getFSDirectory();
+    boolean oldEnabled = fsDirectory.isPosixAclInheritanceEnabled();
+
+    try {
+      fsDirectory.setPosixAclInheritanceEnabled(false);
+      Path dirPath = new Path(path, "dir1");
+      fs.mkdirs(dirPath);
+      AclStatus s = fs.getAclStatus(dirPath);
+      AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
+      assertArrayEquals(new AclEntry[]{
+          aclEntry(ACCESS, USER, "foo", ALL),
+          aclEntry(ACCESS, GROUP, ALL),
+          aclEntry(DEFAULT, USER, ALL),
+          aclEntry(DEFAULT, USER, "foo", ALL),
+          aclEntry(DEFAULT, GROUP, ALL),
+          aclEntry(DEFAULT, MASK, ALL),
+          aclEntry(DEFAULT, OTHER, NONE)}, returned);
+      assertPermission(dirPath, (short) 010750);
+
+      fsDirectory.setPosixAclInheritanceEnabled(true);
+      Path dir2Path = new Path(path, "dir2");
+      fs.mkdirs(dir2Path);
+      AclStatus s2 = fs.getAclStatus(dir2Path);
+      AclEntry[] returned2 = s2.getEntries().toArray(new AclEntry[0]);
+      assertArrayEquals(new AclEntry[]{
+          aclEntry(ACCESS, USER, "foo", ALL),
+          aclEntry(ACCESS, GROUP, ALL),
+          aclEntry(DEFAULT, USER, ALL),
+          aclEntry(DEFAULT, USER, "foo", ALL),
+          aclEntry(DEFAULT, GROUP, ALL),
+          aclEntry(DEFAULT, MASK, ALL),
+          aclEntry(DEFAULT, OTHER, NONE)}, returned2);
+      assertPermission(dir2Path, (short) 010770);
+    } finally {
+      fsDirectory.setPosixAclInheritanceEnabled(oldEnabled);
+      fs.getConf().set(FS_PERMISSIONS_UMASK_KEY, oldUMask);
+    }
+  }
+
   @Test
   public void testOnlyAccessAclNewDir() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));

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

@@ -127,7 +127,7 @@ public class TestGetBlockLocations {
         MOCK_INODE_ID, FILE_NAME.getBytes(StandardCharsets.UTF_8),
         perm, 1, 1, new BlockInfo[] {}, (short) 1,
         DFS_BLOCK_SIZE_DEFAULT);
-    fsn.getFSDirectory().addINode(iip, file);
+    fsn.getFSDirectory().addINode(iip, file, null);
     return fsn;
   }
 

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -1570,7 +1570,9 @@ public class TestRenameWithSnapshots {
     FSDirectory fsdir2 = Mockito.spy(fsdir);
     Mockito.doThrow(new NSQuotaExceededException("fake exception")).when(fsdir2)
         .addLastINode((INodesInPath) Mockito.anyObject(),
-            (INode) Mockito.anyObject(), Mockito.anyBoolean());
+            (INode) Mockito.anyObject(),
+            (FsPermission) Mockito.anyObject(),
+            Mockito.anyBoolean());
     Whitebox.setInternalState(fsn, "dir", fsdir2);
     // rename /test/dir1/foo to /test/dir2/subdir2/foo. 
     // FSDirectory#verifyQuota4Rename will pass since the remaining quota is 2.

+ 1075 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testAclCLIWithPosixAclInheritance.xml

@@ -0,0 +1,1075 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="testConf.xsl"?>
+
+<!--
+   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.
+-->
+
+<configuration>
+  <!-- Normal mode is test. To run just the commands and dump the output
+       to the log, set it to nocompare -->
+  <mode>test</mode>
+
+  <!--  Comparator types:
+           ExactComparator
+           SubstringComparator
+           RegexpComparator
+           TokenComparator
+           -->
+  <tests>
+    <!-- Tests for setfacl and getfacl-->
+    <test>
+      <description>getfacl: basic permissions</description>
+      <test-commands>
+        <command>-fs NAMENODE -touchz /file1</command>
+        <command>-fs NAMENODE -getfacl /file1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /file1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /file1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rw-</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r--</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>getfacl: basic permissions for directory</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -getfacl /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /dir1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r-x</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : Add an ACL</description>
+      <test-commands>
+        <command>-fs NAMENODE -touchz /file1</command>
+        <command>-fs NAMENODE -setfacl -m user:bob:r-- /file1</command>
+        <command>-fs NAMENODE -getfacl /file1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /file1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /file1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rw-</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user:bob:r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>mask::r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r--</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : Add multiple ACLs at once</description>
+      <test-commands>
+        <command>-fs NAMENODE -touchz /file1</command>
+        <command>-fs NAMENODE -setfacl -m user:bob:r--,group:users:r-x /file1</command>
+        <command>-fs NAMENODE -getfacl /file1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /file1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /file1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rw-</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user:bob:r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group:users:r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>mask::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r--</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : Remove an ACL</description>
+      <test-commands>
+        <command>-fs NAMENODE -touchz /file1</command>
+        <command>-fs NAMENODE -setfacl -m user:bob:r--,user:charlie:r-x /file1</command>
+        <command>-fs NAMENODE -setfacl -x user:bob /file1</command>
+        <command>-fs NAMENODE -getfacl /file1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /file1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /file1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rw-</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user:charlie:r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!bob)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : Add default ACL</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m user:bob:r--,group:users:r-x /dir1</command>
+        <command>-fs NAMENODE -setfacl -m default:user:charlie:r-x,default:group:admin:rwx /dir1</command>
+        <command>-fs NAMENODE -getfacl /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /dir1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user:bob:r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group:users:r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>mask::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:user:charlie:r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:group:admin:rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:mask::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:other::r-x</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : Add minimal default ACL</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m default:user::rwx /dir1</command>
+        <command>-fs NAMENODE -getfacl /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /dir1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:other::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!default\:mask)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : try adding default ACL to file</description>
+      <test-commands>
+        <command>-fs NAMENODE -touchz /file1</command>
+        <command>-fs NAMENODE -setfacl -m default:user:charlie:r-x /file1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /file1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>setfacl: Invalid ACL: only directories may have a default ACL</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : Remove one default ACL</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m user:bob:r--,group:users:r-x /dir1</command>
+        <command>-fs NAMENODE -setfacl -m default:user:charlie:r-x,default:group:admin:rwx /dir1</command>
+        <command>-fs NAMENODE -setfacl -x default:user:charlie /dir1</command>
+        <command>-fs NAMENODE -getfacl /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /dir1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user:bob:r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group:users:r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>mask::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:group:admin:rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:mask::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:other::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!default:user:charlie).*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : Remove all default ACL</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m user:bob:r--,group:users:r-x /dir1</command>
+        <command>-fs NAMENODE -setfacl -m default:user:charlie:r-x,default:group:admin:rwx /dir1</command>
+        <command>-fs NAMENODE -setfacl -k /dir1</command>
+        <command>-fs NAMENODE -getfacl /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /dir1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user:bob:r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group:users:r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>mask::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!default).*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : Remove all but base ACLs for a directory</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m user:charlie:r-x,default:group:admin:rwx /dir1</command>
+        <command>-fs NAMENODE -setfacl -b /dir1</command>
+        <command>-fs NAMENODE -getfacl /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /dir1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!charlie).*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!default).*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!admin).*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : Remove all but base ACLs for a file</description>
+      <test-commands>
+        <command>-fs NAMENODE -touchz /file1</command>
+        <command>-fs NAMENODE -setfacl -m user:charlie:r-x,group:admin:rwx /file1</command>
+        <command>-fs NAMENODE -setfacl -b /file1</command>
+        <command>-fs NAMENODE -getfacl /file1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /file1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /file1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rw-</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!charlie).*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!admin).*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : check inherit default ACL to file</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m default:user:charlie:r-x,default:group:admin:rwx /dir1</command>
+        <command>-fs NAMENODE -touchz /dir1/file</command>
+        <command>-fs NAMENODE -getfacl /dir1/file</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /dir1/file</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rw-</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user:charlie:r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group:admin:rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^mask::rw-$</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r--</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!default).*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl : check inherit default ACL to dir</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m default:user:charlie:r-x,default:group:admin:rwx /dir1</command>
+        <command>-fs NAMENODE -mkdir /dir1/dir2</command>
+        <command>-fs NAMENODE -getfacl /dir1/dir2</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /dir1/dir2</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user:charlie:r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group:admin:rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^mask::rwx$</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:user:charlie:r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:group:admin:rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:mask::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:other::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r-x</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>getfacl -R : recursive</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m user:charlie:r-x,group:admin:rwx /dir1</command>
+        <command>-fs NAMENODE -mkdir /dir1/dir2</command>
+        <command>-fs NAMENODE -setfacl -m user:user1:r-x,group:users:rwx /dir1/dir2</command>
+        <command>-fs NAMENODE -getfacl -R /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output># file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:r-x#LF#group::r-x#LF#group:admin:rwx#LF#mask::rwx#LF#other::r-x#LF##LF## file: /dir1/dir2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:user1:r-x#LF#group::r-x#LF#group:users:rwx#LF#mask::rwx#LF#other::r-x#LF##LF#</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl -R : recursive</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -mkdir /dir1/dir2</command>
+        <command>-fs NAMENODE -setfacl -R -m user:charlie:r-x,group:admin:rwx /dir1</command>
+        <command>-fs NAMENODE -getfacl -R /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output># file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:r-x#LF#group::r-x#LF#group:admin:rwx#LF#mask::rwx#LF#other::r-x#LF##LF## file: /dir1/dir2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:r-x#LF#group::r-x#LF#group:admin:rwx#LF#mask::rwx#LF#other::r-x#LF##LF#</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl --set : Set full set of ACLs</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m user:charlie:r-x,group:admin:rwx /dir1</command>
+        <command>-fs NAMENODE -setfacl --set user::rw-,group::r--,other::r--,user:user1:r-x,group:users:rw- /dir1</command>
+        <command>-fs NAMENODE -getfacl /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output># file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rw-#LF#user:user1:r-x#LF#group::r--#LF#group:users:rw-#LF#mask::rwx#LF#other::r--#LF##LF#</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl -x mask : remove mask entry along with other ACL entries</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m user:charlie:r-x,group:admin:rwx /dir1</command>
+        <command>-fs NAMENODE -setfacl -x mask::,user:charlie,group:admin /dir1</command>
+        <command>-fs NAMENODE -getfacl /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output># file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#group::r-x#LF#other::r-x#LF##LF#</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>getfacl: only default ACL</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m default:user:charlie:rwx /dir1</command>
+        <command>-fs NAMENODE -getfacl /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /dir1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:user:charlie:rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:group::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:mask::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:other::r-x</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>getfacl: effective permissions</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m user:charlie:rwx,group::-wx,group:sales:rwx,mask::r-x,default:user:charlie:rwx,default:group::r-x,default:group:sales:rwx,default:mask::rw- /dir1</command>
+        <command>-fs NAMENODE -getfacl /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># file: /dir1</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># owner: USERNAME</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output># group: supergroup</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^user:charlie:rwx\t#effective:r-x$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^group::-wx\t#effective:--x$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^group:sales:rwx\t#effective:r-x$</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>mask::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>other::r-x</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:user::rwx</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^default:user:charlie:rwx\t#effective:rw-$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^default:group::r-x\t#effective:r--$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^default:group:sales:rwx\t#effective:rw-$</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:mask::rw-</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>default:other::r-x</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>ls: display extended acl marker</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /dir1/dir2</command>
+        <command>-fs NAMENODE -setfacl -m user:charlie:rwx,group::-wx,group:sales:rwx,mask::r-x,default:user:charlie:rwx,default:group::r-x,default:group:sales:rwx,default:mask::rw- /dir1/dir2</command>
+        <command>-fs NAMENODE -ls /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^drwxr-xr-x\+( )*-( )*USERNAME( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir1/dir2</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl: recursive modify entries with mix of files and directories</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /dir1</command>
+        <command>-fs NAMENODE -touchz /dir1/file1</command>
+        <command>-fs NAMENODE -mkdir -p /dir1/dir2</command>
+        <command>-fs NAMENODE -touchz /dir1/dir2/file2</command>
+        <command>-fs NAMENODE -setfacl -R -m user:charlie:rwx,default:user:charlie:r-x /dir1</command>
+        <command>-fs NAMENODE -getfacl -R /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output># file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2/file2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rw-#LF#user:charlie:rwx#LF#group::r--#LF#mask::rwx#LF#other::r--#LF##LF## file: /dir1/file1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rw-#LF#user:charlie:rwx#LF#group::r--#LF#mask::rwx#LF#other::r--#LF##LF#</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl: recursive remove entries with mix of files and directories</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /dir1</command>
+        <command>-fs NAMENODE -touchz /dir1/file1</command>
+        <command>-fs NAMENODE -mkdir -p /dir1/dir2</command>
+        <command>-fs NAMENODE -touchz /dir1/dir2/file2</command>
+        <command>-fs NAMENODE -setfacl -R -m user:bob:rwx,user:charlie:rwx,default:user:bob:rwx,default:user:charlie:r-x /dir1</command>
+        <command>-fs NAMENODE -setfacl -R -x user:bob,default:user:bob /dir1</command>
+        <command>-fs NAMENODE -getfacl -R /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output># file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2/file2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rw-#LF#user:charlie:rwx#LF#group::r--#LF#mask::rwx#LF#other::r--#LF##LF## file: /dir1/file1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rw-#LF#user:charlie:rwx#LF#group::r--#LF#mask::rwx#LF#other::r--#LF##LF#</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl: recursive set with mix of files and directories</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /dir1</command>
+        <command>-fs NAMENODE -touchz /dir1/file1</command>
+        <command>-fs NAMENODE -mkdir -p /dir1/dir2</command>
+        <command>-fs NAMENODE -touchz /dir1/dir2/file2</command>
+        <command>-fs NAMENODE -setfacl -R --set user::rwx,user:charlie:rwx,group::r-x,other::r-x,default:user:charlie:r-x /dir1</command>
+        <command>-fs NAMENODE -getfacl -R /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output># file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2/file2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF##LF## file: /dir1/file1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF##LF#</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>copyFromLocal: copying file into a directory with a default ACL</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir1</command>
+        <command>-fs NAMENODE -setfacl -m default:user:charlie:rwx /dir1</command>
+        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data1k /dir1/data1k</command>
+        <command>-fs NAMENODE -getfacl /dir1/data1k</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^# file: /dir1/data1k$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^# owner: USERNAME$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^# group: supergroup$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^user::rw-$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^user:charlie:rwx\t#effective:rw-$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^group::r-x\t#effective:r--$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^mask::rw-$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^other::r--$</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpAcrossOutputComparator</type>
+          <expected-output>.*(?!default).*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+  </tests>
+</configuration>