Browse Source

HDFS-14856. Fetch file ACLs while mounting external store. (#1478)

Ashvin 5 năm trước cách đây
mục cha
commit
fabd41fa48

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

@@ -379,6 +379,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR_DEFAULT = "file:///tmp/";
 
   public static final String DFS_PROVIDED_ALIASMAP_LEVELDB_PATH = "dfs.provided.aliasmap.leveldb.path";
+  public static final String DFS_PROVIDED_ACLS_IMPORT_ENABLED =
+      "dfs.provided.acls.import.enabled";
+  public static final boolean DFS_PROVIDED_ACLS_IMPORT_ENABLED_DEFAULT = false;
 
   public static final String  DFS_LIST_LIMIT = "dfs.ls.limit";
   public static final int     DFS_LIST_LIMIT_DEFAULT = 1000;

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -5357,6 +5357,17 @@
     </description>
   </property>
 
+  <property>
+  <name>dfs.provided.acls.import.enabled</name>
+    <value>false</value>
+    <description>
+      Set to true to inherit ACLs (Access Control Lists) from remote stores
+      during mount. Disabled by default, i.e., ACLs are not inherited from
+      remote stores. Note had HDFS ACLs have to be enabled
+      (dfs.namenode.acls.enabled must be set to true) for this to take effect.
+    </description>
+  </property>
+
   <property>
     <name>dfs.provided.aliasmap.load.retries</name>
     <value>0</value>

+ 42 - 12
hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java

@@ -29,6 +29,14 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ACLS_IMPORT_ENABLED;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ACLS_IMPORT_ENABLED_DEFAULT;
 
 /**
  * Traversal of an external FileSystem.
@@ -37,12 +45,28 @@ import org.apache.hadoop.fs.Path;
 @InterfaceStability.Unstable
 public class FSTreeWalk extends TreeWalk {
 
+  public static final Logger LOG =
+      LoggerFactory.getLogger(FSTreeWalk.class);
+
   private final Path root;
   private final FileSystem fs;
+  private final boolean enableACLs;
 
   public FSTreeWalk(Path root, Configuration conf) throws IOException {
     this.root = root;
     fs = root.getFileSystem(conf);
+
+    boolean mountACLsEnabled = conf.getBoolean(DFS_PROVIDED_ACLS_IMPORT_ENABLED,
+        DFS_PROVIDED_ACLS_IMPORT_ENABLED_DEFAULT);
+    boolean localACLsEnabled = conf.getBoolean(DFS_NAMENODE_ACLS_ENABLED_KEY,
+        DFS_NAMENODE_ACLS_ENABLED_DEFAULT);
+    if (!localACLsEnabled && mountACLsEnabled) {
+      LOG.warn("Mount ACLs have been enabled but HDFS ACLs are not. " +
+          "Disabling ACLs on the mount {}", root);
+      this.enableACLs = false;
+    } else {
+      this.enableACLs = mountACLsEnabled;
+    }
   }
 
   @Override
@@ -55,7 +79,8 @@ public class FSTreeWalk extends TreeWalk {
     try {
       ArrayList<TreePath> ret = new ArrayList<>();
       for (FileStatus s : fs.listStatus(path.getFileStatus().getPath())) {
-        ret.add(new TreePath(s, id, i, fs));
+        AclStatus aclStatus = getAclStatus(fs, s.getPath());
+        ret.add(new TreePath(s, id, i, fs, aclStatus));
       }
       return ret;
     } catch (FileNotFoundException e) {
@@ -71,20 +96,19 @@ public class FSTreeWalk extends TreeWalk {
     }
 
     FSTreeIterator(TreePath p) {
-      getPendingQueue().addFirst(
-          new TreePath(p.getFileStatus(), p.getParentId(), this, fs));
+      this(p.getFileStatus(), p.getParentId());
     }
 
-    FSTreeIterator(Path p) throws IOException {
+    FSTreeIterator(FileStatus fileStatus, long parentId) {
+      Path path = fileStatus.getPath();
+      AclStatus acls;
       try {
-        FileStatus s = fs.getFileStatus(root);
-        getPendingQueue().addFirst(new TreePath(s, -1L, this, fs));
-      } catch (FileNotFoundException e) {
-        if (p.equals(root)) {
-          throw e;
-        }
-        throw new ConcurrentModificationException("FS modified");
+        acls = getAclStatus(fs, path);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
       }
+      TreePath treePath = new TreePath(fileStatus, parentId, this, fs, acls);
+      getPendingQueue().addFirst(treePath);
     }
 
     @Override
@@ -97,10 +121,16 @@ public class FSTreeWalk extends TreeWalk {
 
   }
 
+  private AclStatus getAclStatus(FileSystem fileSystem, Path path)
+      throws IOException {
+    return enableACLs ? fileSystem.getAclStatus(path) : null;
+  }
+
   @Override
   public TreeIterator iterator() {
     try {
-      return new FSTreeIterator(root);
+      FileStatus s = fs.getFileStatus(root);
+      return new FSTreeIterator(s, -1L);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }

+ 2 - 3
hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java

@@ -23,7 +23,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.security.UserGroupInformation;
 
 /**
@@ -73,12 +72,12 @@ public class SingleUGIResolver extends UGIResolver implements Configurable {
   }
 
   @Override
-  public String user(FileStatus s) {
+  public String user(String s) {
     return user;
   }
 
   @Override
-  public String group(FileStatus s) {
+  public String group(String s) {
     return group;
   }
 

+ 36 - 11
hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -27,6 +28,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.PathHandle;
+import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
@@ -52,23 +54,38 @@ public class TreePath {
   private final FileStatus stat;
   private final TreeWalk.TreeIterator i;
   private final FileSystem fs;
+  private final AclStatus acls;
 
-  protected TreePath(FileStatus stat, long parentId, TreeWalk.TreeIterator i,
-      FileSystem fs) {
+  @VisibleForTesting
+  public TreePath(FileStatus stat, long parentId, TreeWalk.TreeIterator i) {
+    this(stat, parentId, i, null, null);
+  }
+
+  public TreePath(FileStatus stat, long parentId, TreeWalk.TreeIterator i,
+      FileSystem fs, AclStatus acls) {
     this.i = i;
     this.stat = stat;
     this.parentId = parentId;
     this.fs = fs;
+    this.acls = acls;
   }
 
   public FileStatus getFileStatus() {
     return stat;
   }
 
+  public AclStatus getAclStatus() {
+    return acls;
+  }
+
   public long getParentId() {
     return parentId;
   }
 
+  public TreeWalk.TreeIterator getIterator() {
+    return i;
+  }
+
   public long getId() {
     if (id < 0) {
       throw new IllegalStateException();
@@ -76,8 +93,8 @@ public class TreePath {
     return id;
   }
 
-  void accept(long id) {
-    this.id = id;
+  public void accept(long pathId) {
+    this.id = pathId;
     i.onAccept(this, id);
   }
 
@@ -121,14 +138,14 @@ public class TreePath {
   INode toFile(UGIResolver ugi, BlockResolver blk,
       BlockAliasMap.Writer<FileRegion> out) throws IOException {
     final FileStatus s = getFileStatus();
-    ugi.addUser(s.getOwner());
-    ugi.addGroup(s.getGroup());
+    final AclStatus aclStatus = getAclStatus();
+    long permissions = ugi.getPermissionsProto(s, aclStatus);
     INodeFile.Builder b = INodeFile.newBuilder()
         .setReplication(blk.getReplication(s))
         .setModificationTime(s.getModificationTime())
         .setAccessTime(s.getAccessTime())
         .setPreferredBlockSize(blk.preferredBlockSize(s))
-        .setPermission(ugi.resolve(s))
+        .setPermission(permissions)
         .setStoragePolicyID(HdfsConstants.PROVIDED_STORAGE_POLICY_ID);
 
     // pathhandle allows match as long as the file matches exactly.
@@ -141,7 +158,11 @@ public class TreePath {
             "Exact path handle not supported by filesystem " + fs.toString());
       }
     }
-    // TODO: storage policy should be configurable per path; use BlockResolver
+    if (aclStatus != null) {
+      throw new UnsupportedOperationException(
+          "ACLs not supported by ImageWriter");
+    }
+    //TODO: storage policy should be configurable per path; use BlockResolver
     long off = 0L;
     for (BlockProto block : blk.resolve(s)) {
       b.addBlocks(block);
@@ -159,13 +180,17 @@ public class TreePath {
 
   INode toDirectory(UGIResolver ugi) {
     final FileStatus s = getFileStatus();
-    ugi.addUser(s.getOwner());
-    ugi.addGroup(s.getGroup());
+    final AclStatus aclStatus = getAclStatus();
+    long permissions = ugi.getPermissionsProto(s, aclStatus);
     INodeDirectory.Builder b = INodeDirectory.newBuilder()
         .setModificationTime(s.getModificationTime())
         .setNsQuota(DEFAULT_NAMESPACE_QUOTA)
         .setDsQuota(DEFAULT_STORAGE_SPACE_QUOTA)
-        .setPermission(ugi.resolve(s));
+        .setPermission(permissions);
+    if (aclStatus != null) {
+      throw new UnsupportedOperationException(
+          "ACLs not supported by ImageWriter");
+    }
     INode.Builder ib = INode.newBuilder()
         .setType(INode.Type.DIRECTORY)
         .setId(id)

+ 1 - 1
hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java

@@ -50,7 +50,7 @@ public abstract class TreeWalk implements Iterable<TreePath> {
 
     private final Deque<TreePath> pending;
 
-    TreeIterator() {
+    public TreeIterator() {
       this(new ArrayDeque<TreePath>());
     }
 

+ 71 - 10
hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/UGIResolver.java

@@ -24,6 +24,9 @@ import java.util.Map;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 
 /**
@@ -34,9 +37,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 @InterfaceStability.Unstable
 public abstract class UGIResolver {
 
-  static final int USER_STRID_OFFSET = 40;
-  static final int GROUP_STRID_OFFSET = 16;
-  static final long USER_GROUP_STRID_MASK = (1 << 24) - 1;
+  public static final int USER_STRID_OFFSET = 40;
+  public static final int GROUP_STRID_OFFSET = 16;
+  public static final long USER_GROUP_STRID_MASK = (1 << 24) - 1;
 
   /**
    * Permission is serialized as a 64-bit long. [0:24):[25:48):[48:64) (in Big
@@ -117,19 +120,77 @@ public abstract class UGIResolver {
   }
 
   public long resolve(FileStatus s) {
-    return buildPermissionStatus(user(s), group(s), permission(s).toShort());
+    String resolvedGroup = group(s.getGroup());
+    String resolvedOwner = user(s.getOwner());
+    FsPermission resolvedPermission = permission(s.getPermission());
+    return buildPermissionStatus(
+        resolvedOwner, resolvedGroup, resolvedPermission.toShort());
   }
 
-  public String user(FileStatus s) {
-    return s.getOwner();
+  private long resolve(AclStatus aclStatus) {
+    String resolvedOwner = user(aclStatus.getOwner());
+    String resolvedGroup = group(aclStatus.getGroup());
+    FsPermission resolvedPermision = permission(aclStatus.getPermission());
+    return buildPermissionStatus(
+        resolvedOwner, resolvedGroup, resolvedPermision.toShort());
   }
 
-  public String group(FileStatus s) {
-    return s.getGroup();
+  protected String user(String s) {
+    return s;
   }
 
-  public FsPermission permission(FileStatus s) {
-    return s.getPermission();
+  protected String group(String s) {
+    return s;
   }
 
+  public FsPermission permission(FsPermission s) {
+    return s;
+  }
+
+  /**
+   * Get the serialized, local permissions for the external
+   * {@link FileStatus} or {@link AclStatus}. {@code remoteAcl} is used when it
+   * is not null, otherwise {@code remoteStatus} is used.
+   *
+   * @param remoteStatus FileStatus on remote store.
+   * @param remoteAcl AclStatus on external store.
+   * @return serialized, local permissions the FileStatus or AclStatus map to.
+   */
+  public long getPermissionsProto(FileStatus remoteStatus,
+      AclStatus remoteAcl) {
+    addUGI(remoteStatus, remoteAcl);
+    if (remoteAcl == null) {
+      return resolve(remoteStatus);
+    } else {
+      return resolve(remoteAcl);
+    }
+  }
+
+  /**
+   * Add the users and groups specified by the given {@link FileStatus} and
+   * {@link AclStatus}.
+   *
+   * @param remoteStatus
+   * @param remoteAcl
+   */
+  private void addUGI(FileStatus remoteStatus, AclStatus remoteAcl) {
+    if (remoteAcl != null) {
+      addUser(remoteAcl.getOwner());
+      addGroup(remoteAcl.getGroup());
+      for (AclEntry entry : remoteAcl.getEntries()) {
+        // add the users and groups in this acl entry to ugi
+        String name = entry.getName();
+        if (name != null) {
+          if (entry.getType() == AclEntryType.USER) {
+            addUser(name);
+          } else if (entry.getType() == AclEntryType.GROUP) {
+            addGroup(name);
+          }
+        }
+      }
+    } else {
+      addUser(remoteStatus.getOwner());
+      addGroup(remoteStatus.getGroup());
+    }
+  }
 }

+ 3 - 3
hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java

@@ -95,7 +95,7 @@ public class RandomTreeWalk extends TreeWalk {
     int nChildren = r.nextInt(children);
     ArrayList<TreePath> ret = new ArrayList<TreePath>();
     for (int i = 0; i < nChildren; ++i) {
-      ret.add(new TreePath(genFileStatus(p, r), p.getId(), walk, null));
+      ret.add(new TreePath(genFileStatus(p, r), p.getId(), walk));
     }
     return ret;
   }
@@ -163,12 +163,12 @@ public class RandomTreeWalk extends TreeWalk {
     RandomTreeIterator(long seed) {
       Random r = new Random(seed);
       FileStatus iroot = genFileStatus(null, r);
-      getPendingQueue().addFirst(new TreePath(iroot, -1, this, null));
+      getPendingQueue().addFirst(new TreePath(iroot, -1, this));
     }
 
     RandomTreeIterator(TreePath p) {
       getPendingQueue().addFirst(
-          new TreePath(p.getFileStatus(), p.getParentId(), this, null));
+          new TreePath(p.getFileStatus(), p.getParentId(), this));
     }
 
     @Override

+ 120 - 0
hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSTreeWalk.java

@@ -0,0 +1,120 @@
+/**
+ * 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.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Validate FSTreeWalk specific behavior.
+ */
+public class TestFSTreeWalk {
+  /**
+   * Verify that the ACLs are fetched when configured.
+   */
+  @Test
+  public void testImportAcl() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ACLS_IMPORT_ENABLED, true);
+
+    FileSystem fs = mock(FileSystem.class);
+    Path root = mock(Path.class);
+    when(root.getFileSystem(conf)).thenReturn(fs);
+
+    Map<Path, FileStatus> expectedChildren = new HashMap<>();
+    FileStatus child1 = new FileStatus(0, true, 0, 0, 1, new Path("/a"));
+    FileStatus child2 = new FileStatus(0, true, 0, 0, 1, new Path("/b"));
+    expectedChildren.put(child1.getPath(), child1);
+    expectedChildren.put(child2.getPath(), child2);
+    when(fs.listStatus(root))
+        .thenReturn(expectedChildren.values().toArray(new FileStatus[1]));
+
+    AclStatus expectedAcls = mock(AclStatus.class);
+    when(fs.getAclStatus(any(Path.class))).thenReturn(expectedAcls);
+
+    FSTreeWalk fsTreeWalk = new FSTreeWalk(root, conf);
+
+    FileStatus rootFileStatus = new FileStatus(0, true, 0, 0, 1, root);
+    TreePath treePath = new TreePath(rootFileStatus, 1, null);
+
+    Iterable<TreePath> result = fsTreeWalk.getChildren(treePath, 1, null);
+    for (TreePath path : result) {
+      FileStatus expectedChildStatus
+          = expectedChildren.remove(path.getFileStatus().getPath());
+      assertNotNull(expectedChildStatus);
+
+      AclStatus childAcl = path.getAclStatus();
+      assertEquals(expectedAcls, childAcl);
+    }
+
+    assertEquals(0, expectedChildren.size());
+  }
+
+  /**
+   * Verify ACL enabled TreeWalk iterator throws an error if the external file
+   * system does not support ACLs.
+   */
+  @Test(expected = UnsupportedOperationException.class)
+  public void testACLNotSupported() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ACLS_IMPORT_ENABLED, true);
+
+    FileSystem fs = mock(FileSystem.class);
+    when(fs.getAclStatus(any())).thenThrow(new UnsupportedOperationException());
+    Path root = mock(Path.class);
+    when(root.getFileSystem(conf)).thenReturn(fs);
+    FileStatus rootFileStatus = new FileStatus(0, true, 0, 0, 1, root);
+    when(fs.getFileStatus(root)).thenReturn(rootFileStatus);
+
+    FSTreeWalk fsTreeWalk = new FSTreeWalk(root, conf);
+    TreeWalk.TreeIterator iter = fsTreeWalk.iterator();
+    fail("Unexpected successful creation of iter: " + iter);
+  }
+
+  /**
+   * Verify creation of INode for ACL enabled TreePath throws an error.
+   */
+  @Test(expected = UnsupportedOperationException.class)
+  public void testToINodeACLNotSupported() throws Exception {
+    BlockResolver blockResolver = new FixedBlockResolver();
+    Path root = new Path("/");
+    FileStatus rootFileStatus = new FileStatus(0, false, 0, 0, 1, root);
+
+    AclStatus acls = mock(AclStatus.class);
+    TreePath treePath = new TreePath(rootFileStatus, 1, null, null, acls);
+
+    UGIResolver ugiResolver = mock(UGIResolver.class);
+    when(ugiResolver.getPermissionsProto(null, acls)).thenReturn(1L);
+    treePath.toINode(ugiResolver, blockResolver, null);
+  }
+}

+ 32 - 0
hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSingleUGIResolver.java

@@ -23,6 +23,11 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryScope;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -93,6 +98,33 @@ public class TestSingleUGIResolver {
     assertEquals(user, ids.get(1));
   }
 
+  @Test
+  public void testAclResolution() {
+    long perm;
+
+    FsPermission p1 = new FsPermission((short)0755);
+    FileStatus fileStatus = file("dingo", "dingo", p1);
+    perm = ugi.getPermissionsProto(fileStatus, null);
+    match(perm, p1);
+
+    AclEntry aclEntry = new AclEntry.Builder()
+        .setType(AclEntryType.USER)
+        .setScope(AclEntryScope.ACCESS)
+        .setPermission(FsAction.ALL)
+        .setName("dingo")
+        .build();
+
+    AclStatus aclStatus = new AclStatus.Builder()
+        .owner("dingo")
+        .group(("dingo"))
+        .addEntry(aclEntry)
+        .setPermission(p1)
+        .build();
+
+    perm = ugi.getPermissionsProto(null, aclStatus);
+    match(perm, p1);
+  }
+
   @Test(expected=IllegalArgumentException.class)
   public void testInvalidUid() {
     Configuration conf = ugi.getConf();