You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by we...@apache.org on 2020/06/16 22:59:33 UTC

[hadoop] branch trunk updated: HDFS-15372. Files in snapshots no longer see attribute provider permissions. Contributed by Stephen O'Donnell.

This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 730a39d  HDFS-15372. Files in snapshots no longer see attribute provider permissions. Contributed by Stephen O'Donnell.
730a39d is described below

commit 730a39d1388548f22f76132a6734d61c24c3eb72
Author: Stephen O'Donnell <so...@apache.org>
AuthorDate: Tue Jun 16 15:58:16 2020 -0700

    HDFS-15372. Files in snapshots no longer see attribute provider permissions. Contributed by Stephen O'Donnell.
    
    Signed-off-by: Wei-Chiu Chuang <we...@apache.org>
---
 .../hadoop/hdfs/server/namenode/FSDirectory.java   |  16 ++-
 .../hdfs/server/namenode/FSPermissionChecker.java  |  46 +++++----
 .../hadoop/hdfs/server/namenode/INodesInPath.java  |  42 ++++++++
 .../namenode/TestINodeAttributeProvider.java       | 115 +++++++++++++++++++++
 4 files changed, 199 insertions(+), 20 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 5895c6b..cd9eb09 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -73,7 +73,6 @@ import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -2032,7 +2031,20 @@ public class FSDirectory implements Closeable {
       // first empty component for the root.  however file status
       // related calls are expected to strip out the root component according
       // to TestINodeAttributeProvider.
-      byte[][] components = iip.getPathComponents();
+      // Due to HDFS-15372 the attribute provider should received the resolved
+      // snapshot path. Ie, rather than seeing /d/.snapshot/sn/data it should
+      // see /d/data. However, for the path /d/.snapshot/sn it should see this
+      // full path. Node.getPathComponents always resolves the path to the
+      // original location, so we need to check if ".snapshot/sn" is the last
+      // path to ensure the provider receives the correct components.
+      byte[][] components;
+      if (iip.isSnapshot() && !iip.isDotSnapshotDirPrefix()) {
+        // For snapshot paths, node.getPathComponents unless the last component
+        // is like ".snapshot/sn"
+        components = node.getPathComponents();
+      } else {
+        components = iip.getPathComponents();
+      }
       components = Arrays.copyOfRange(components, 1, components.length);
       nodeAttrs = ap.getAttributes(components, nodeAttrs);
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
index c697ead7..615b164 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
@@ -19,11 +19,14 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.Stack;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.ipc.CallerContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -207,7 +210,7 @@ public class FSPermissionChecker implements AccessControlEnforcer {
     final INodeAttributes[] inodeAttrs = new INodeAttributes[inodes.length];
     final byte[][] components = inodesInPath.getPathComponents();
     for (int i = 0; i < inodes.length && inodes[i] != null; i++) {
-      inodeAttrs[i] = getINodeAttrs(components, i, inodes[i], snapshotId);
+      inodeAttrs[i] = getINodeAttrs(inodes[i], snapshotId);
     }
 
     String path = inodesInPath.getPath();
@@ -257,8 +260,7 @@ public class FSPermissionChecker implements AccessControlEnforcer {
   void checkPermission(INode inode, int snapshotId, FsAction access)
       throws AccessControlException {
     byte[][] pathComponents = inode.getPathComponents();
-    INodeAttributes nodeAttributes = getINodeAttrs(pathComponents,
-        pathComponents.length - 1, inode, snapshotId);
+    INodeAttributes nodeAttributes = getINodeAttrs(inode, snapshotId);
     try {
       INodeAttributes[] iNodeAttr = {nodeAttributes};
       AccessControlEnforcer enforcer = getAccessControlEnforcer();
@@ -367,23 +369,31 @@ public class FSPermissionChecker implements AccessControlEnforcer {
         authzContext.getSubAccess(), authzContext.isIgnoreEmptyDir());
   }
 
-  private INodeAttributes getINodeAttrs(byte[][] pathByNameArr, int pathIdx,
-      INode inode, int snapshotId) {
+  private INodeAttributes getINodeAttrs(INode inode, int snapshotId) {
     INodeAttributes inodeAttrs = inode.getSnapshotINode(snapshotId);
+    /**
+     * This logic is similar to {@link FSDirectory#getAttributes()} and it
+     * ensures that the attribute provider sees snapshot paths resolved to their
+     * original location. This means the attributeProvider can apply permissions
+     * to the snapshot paths in the same was as the live paths. See HDFS-15372.
+     */
     if (getAttributesProvider() != null) {
-      String[] elements = new String[pathIdx + 1];
       /**
-       * {@link INode#getPathComponents(String)} returns a null component
-       * for the root only path "/". Assign an empty string if so.
+       * If we have an inode representing a path like /d/.snapshot/snap1
+       * then calling inode.getPathComponents returns [null, d, snap1]. If we
+       * call inode.getFullPathName() it will return /d/.snapshot/snap1. For
+       * this special path (snapshot root) the attribute provider should see:
+       *
+       * [null, d, .snapshot/snap1]
+       *
+       * Using IIP.resolveFromRoot, it will take the inode fullPathName and
+       * construct an IIP object that give the correct components as above.
        */
-      if (pathByNameArr.length == 1 && pathByNameArr[0] == null) {
-        elements[0] = "";
-      } else {
-        for (int i = 0; i < elements.length; i++) {
-          elements[i] = DFSUtil.bytes2String(pathByNameArr[i]);
-        }
-      }
-      inodeAttrs = getAttributesProvider().getAttributes(elements, inodeAttrs);
+      INodesInPath iip = INodesInPath.resolveFromRoot(inode);
+      byte[][] components = iip.getPathComponents();
+      components = Arrays.copyOfRange(components, 1, components.length);
+      inodeAttrs = getAttributesProvider()
+          .getAttributes(components, inodeAttrs);
     }
     return inodeAttrs;
   }
@@ -439,7 +449,7 @@ public class FSPermissionChecker implements AccessControlEnforcer {
       if (!(cList.isEmpty() && ignoreEmptyDir)) {
         //TODO have to figure this out with inodeattribute provider
         INodeAttributes inodeAttr =
-            getINodeAttrs(components, pathIdx, d, snapshotId);
+            getINodeAttrs(d, snapshotId);
         if (!hasPermission(inodeAttr, access)) {
           throw new AccessControlException(
               toAccessControlString(inodeAttr, d.getFullPathName(), access));
@@ -457,7 +467,7 @@ public class FSPermissionChecker implements AccessControlEnforcer {
         if (inodeAttr.getFsPermission().getStickyBit()) {
           for (INode child : cList) {
             INodeAttributes childInodeAttr =
-                getINodeAttrs(components, pathIdx, child, snapshotId);
+                getINodeAttrs(child, snapshotId);
             if (isStickyBitViolated(inodeAttr, childInodeAttr)) {
               List<byte[]> allComponentList = new ArrayList<>();
               for (int i = 0; i <= pathIdx; ++i) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
index f072220..179c1c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
@@ -46,6 +46,20 @@ public class INodesInPath {
         Arrays.equals(HdfsServerConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
   }
 
+  /**
+   * Returns true if the given path component starts with the same byte stream
+   * as {@link HdfsConstants#DOT_SNAPSHOT_DIR}, indicating the component
+   * starts with a DotSnapshot directory.
+   * @param pathComponent Bytes representing the pathComponent
+   * @return True is the component starts with
+   *         {@link HdfsConstants#DOT_SNAPSHOT_DIR} and false otherwise.
+   */
+  private static boolean isDotSnapshotDirPrefix(byte[] pathComponent) {
+    return pathComponent != null &&
+        isDotSnapshotDir(Arrays.copyOf(
+            pathComponent, HdfsServerConstants.DOT_SNAPSHOT_DIR_BYTES.length));
+  }
+
   private static INode[] getINodes(final INode inode) {
     int depth = 0, index;
     INode tmp = inode;
@@ -135,6 +149,27 @@ public class INodesInPath {
     return resolve(startingDir, components, false);
   }
 
+  /**
+   * Retrieves the existing INodes from a path, starting at the root directory.
+   * The root directory is located by following the parent link in the inode
+   * recursively until the final root inode is found.
+   * The inodes returned will depend upon the output of inode.getFullPathName().
+   * For a snapshot path, like /data/.snapshot/snap1, it will be resolved to:
+   *     [null, data, .snapshot/snap1]
+   * For a file in the snapshot, as inode.getFullPathName resolves the snapshot
+   * information, the returned inodes for a path like /data/.snapshot/snap1/d1
+   * would be:
+   *     [null, data, d1]
+   * @param inode the {@link INode} to be resolved
+   * @return INodesInPath
+   */
+  static INodesInPath resolveFromRoot(INode inode) {
+    INode[] inodes = getINodes(inode);
+    byte[][] paths = INode.getPathComponents(inode.getFullPathName());
+    INodeDirectory rootDir = inodes[0].asDirectory();
+    return resolve(rootDir, paths);
+  }
+
   static INodesInPath resolve(final INodeDirectory startingDir,
       byte[][] components, final boolean isRaw) {
     Preconditions.checkArgument(startingDir.compareTo(components[0]) == 0);
@@ -463,6 +498,13 @@ public class INodesInPath {
   }
 
   /**
+   * @return Return true if .snapshot is the prefix of the last path component.
+   */
+  boolean isDotSnapshotDirPrefix() {
+    return isDotSnapshotDirPrefix(getLastLocalName());
+  }
+
+  /**
    * @return if this is a /.reserved/raw path.
    */
   public boolean isRaw() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
index 433be79..e7e1f90 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.AccessControlException;
@@ -80,6 +81,7 @@ public class TestINodeAttributeProvider {
               ancestorAccess, parentAccess, access, subAccess, ignoreEmptyDir);
         }
         CALLED.add("checkPermission|" + ancestorAccess + "|" + parentAccess + "|" + access);
+        CALLED.add("checkPermission|" + path);
       }
 
       @Override
@@ -93,6 +95,7 @@ public class TestINodeAttributeProvider {
         CALLED.add("checkPermission|" + authzContext.getAncestorAccess()
             + "|" + authzContext.getParentAccess() + "|" + authzContext
             .getAccess());
+        CALLED.add("checkPermission|" + authzContext.getPath());
       }
     }
 
@@ -109,7 +112,12 @@ public class TestINodeAttributeProvider {
     @Override
     public INodeAttributes getAttributes(String[] pathElements,
         final INodeAttributes inode) {
+      String fullPath = String.join("/", pathElements);
+      if (!fullPath.startsWith("/")) {
+        fullPath = "/" + fullPath;
+      }
       CALLED.add("getAttributes");
+      CALLED.add("getAttributes|"+fullPath);
       final boolean useDefault = useDefault(pathElements);
       final boolean useNullAcl = useNullAclFeature(pathElements);
       return new INodeAttributes() {
@@ -485,4 +493,111 @@ public class TestINodeAttributeProvider {
       }
     });
   }
+
+  @Test
+  // HDFS-15372 - Attribute provider should not see the snapshot path as it
+  // should be resolved into the original path name before it hits the provider.
+  public void testAttrProviderSeesResolvedSnapahotPaths() throws Exception {
+    FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+    DistributedFileSystem hdfs = miniDFS.getFileSystem();
+    final Path userPath = new Path("/user");
+    final Path authz = new Path("/user/authz");
+    final Path authzChild = new Path("/user/authz/child2");
+
+    fs.mkdirs(userPath);
+    fs.setPermission(userPath, new FsPermission(HDFS_PERMISSION));
+    fs.mkdirs(authz);
+    hdfs.allowSnapshot(userPath);
+    fs.setPermission(authz, new FsPermission(HDFS_PERMISSION));
+    fs.mkdirs(authzChild);
+    fs.setPermission(authzChild, new FsPermission(HDFS_PERMISSION));
+    fs.createSnapshot(userPath, "snapshot_1");
+    UserGroupInformation ugi = UserGroupInformation.createUserForTesting("u1",
+        new String[]{"g1"});
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+        final Path snapChild =
+            new Path("/user/.snapshot/snapshot_1/authz/child2");
+        // Run various methods on the path to access the attributes etc.
+        fs.getAclStatus(snapChild);
+        fs.getContentSummary(snapChild);
+        fs.getFileStatus(snapChild);
+        Assert.assertFalse(CALLED.contains("getAttributes|" +
+            snapChild.toString()));
+        Assert.assertTrue(CALLED.contains("getAttributes|/user/authz/child2"));
+        // The snapshot path should be seen by the permission checker, but when
+        // it checks access, the paths will be resolved so the attributeProvider
+        // only sees the resolved path.
+        Assert.assertTrue(
+            CALLED.contains("checkPermission|" + snapChild.toString()));
+        CALLED.clear();
+        fs.getAclStatus(new Path("/"));
+        Assert.assertTrue(CALLED.contains("checkPermission|/"));
+        Assert.assertTrue(CALLED.contains("getAttributes|/"));
+
+        CALLED.clear();
+        fs.getFileStatus(new Path("/user"));
+        Assert.assertTrue(CALLED.contains("checkPermission|/user"));
+        Assert.assertTrue(CALLED.contains("getAttributes|/user"));
+
+        CALLED.clear();
+        fs.getFileStatus(new Path("/user/.snapshot"));
+        Assert.assertTrue(CALLED.contains("checkPermission|/user/.snapshot"));
+        // attribute provider never sees the .snapshot path directly.
+        Assert.assertFalse(CALLED.contains("getAttributes|/user/.snapshot"));
+
+        CALLED.clear();
+        fs.getFileStatus(new Path("/user/.snapshot/snapshot_1"));
+        Assert.assertTrue(
+            CALLED.contains("checkPermission|/user/.snapshot/snapshot_1"));
+        Assert.assertTrue(
+            CALLED.contains("getAttributes|/user/.snapshot/snapshot_1"));
+
+        CALLED.clear();
+        fs.getFileStatus(new Path("/user/.snapshot/snapshot_1/authz"));
+        Assert.assertTrue(CALLED
+            .contains("checkPermission|/user/.snapshot/snapshot_1/authz"));
+        Assert.assertTrue(CALLED.contains("getAttributes|/user/authz"));
+
+        CALLED.clear();
+        fs.getFileStatus(new Path("/user/authz"));
+        Assert.assertTrue(CALLED.contains("checkPermission|/user/authz"));
+        Assert.assertTrue(CALLED.contains("getAttributes|/user/authz"));
+        return null;
+      }
+    });
+    // Delete the files / folders covered by the snapshot, then re-check they
+    // are all readable correctly.
+    fs.delete(authz, true);
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+
+        CALLED.clear();
+        fs.getFileStatus(new Path("/user/.snapshot"));
+        Assert.assertTrue(CALLED.contains("checkPermission|/user/.snapshot"));
+        // attribute provider never sees the .snapshot path directly.
+        Assert.assertFalse(CALLED.contains("getAttributes|/user/.snapshot"));
+
+        CALLED.clear();
+        fs.getFileStatus(new Path("/user/.snapshot/snapshot_1"));
+        Assert.assertTrue(
+            CALLED.contains("checkPermission|/user/.snapshot/snapshot_1"));
+        Assert.assertTrue(
+            CALLED.contains("getAttributes|/user/.snapshot/snapshot_1"));
+
+        CALLED.clear();
+        fs.getFileStatus(new Path("/user/.snapshot/snapshot_1/authz"));
+        Assert.assertTrue(CALLED
+            .contains("checkPermission|/user/.snapshot/snapshot_1/authz"));
+        Assert.assertTrue(CALLED.contains("getAttributes|/user/authz"));
+
+        return null;
+      }
+    });
+
+  }
 }


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