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/18 13:45:49 UTC

[hadoop] 01/02: Revert "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 branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 10880dc396a6147efb75dc2566710b572f5f57eb
Author: Wei-Chiu Chuang <we...@apache.org>
AuthorDate: Thu Jun 18 06:43:28 2020 -0700

    Revert "HDFS-15372. Files in snapshots no longer see attribute provider permissions. Contributed by Stephen O'Donnell."
    
    This reverts commit 0b9e5ea592b66e1b370feaae9677a7b99fdbd03c.
---
 .../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, 20 insertions(+), 199 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 34ee959..7eae564 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,6 +73,7 @@ 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;
@@ -2021,20 +2022,7 @@ 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.
-      // 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();
-      }
+      byte[][] 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 615b164..c697ead7 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,14 +19,11 @@ 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;
@@ -210,7 +207,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(inodes[i], snapshotId);
+      inodeAttrs[i] = getINodeAttrs(components, i, inodes[i], snapshotId);
     }
 
     String path = inodesInPath.getPath();
@@ -260,7 +257,8 @@ public class FSPermissionChecker implements AccessControlEnforcer {
   void checkPermission(INode inode, int snapshotId, FsAction access)
       throws AccessControlException {
     byte[][] pathComponents = inode.getPathComponents();
-    INodeAttributes nodeAttributes = getINodeAttrs(inode, snapshotId);
+    INodeAttributes nodeAttributes = getINodeAttrs(pathComponents,
+        pathComponents.length - 1, inode, snapshotId);
     try {
       INodeAttributes[] iNodeAttr = {nodeAttributes};
       AccessControlEnforcer enforcer = getAccessControlEnforcer();
@@ -369,31 +367,23 @@ public class FSPermissionChecker implements AccessControlEnforcer {
         authzContext.getSubAccess(), authzContext.isIgnoreEmptyDir());
   }
 
-  private INodeAttributes getINodeAttrs(INode inode, int snapshotId) {
+  private INodeAttributes getINodeAttrs(byte[][] pathByNameArr, int pathIdx,
+      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];
       /**
-       * 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.
+       * {@link INode#getPathComponents(String)} returns a null component
+       * for the root only path "/". Assign an empty string if so.
        */
-      INodesInPath iip = INodesInPath.resolveFromRoot(inode);
-      byte[][] components = iip.getPathComponents();
-      components = Arrays.copyOfRange(components, 1, components.length);
-      inodeAttrs = getAttributesProvider()
-          .getAttributes(components, inodeAttrs);
+      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);
     }
     return inodeAttrs;
   }
@@ -449,7 +439,7 @@ public class FSPermissionChecker implements AccessControlEnforcer {
       if (!(cList.isEmpty() && ignoreEmptyDir)) {
         //TODO have to figure this out with inodeattribute provider
         INodeAttributes inodeAttr =
-            getINodeAttrs(d, snapshotId);
+            getINodeAttrs(components, pathIdx, d, snapshotId);
         if (!hasPermission(inodeAttr, access)) {
           throw new AccessControlException(
               toAccessControlString(inodeAttr, d.getFullPathName(), access));
@@ -467,7 +457,7 @@ public class FSPermissionChecker implements AccessControlEnforcer {
         if (inodeAttr.getFsPermission().getStickyBit()) {
           for (INode child : cList) {
             INodeAttributes childInodeAttr =
-                getINodeAttrs(child, snapshotId);
+                getINodeAttrs(components, pathIdx, 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 179c1c0..f072220 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,20 +46,6 @@ 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;
@@ -149,27 +135,6 @@ 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);
@@ -498,13 +463,6 @@ 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 e7e1f90..433be79 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,7 +33,6 @@ 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;
@@ -81,7 +80,6 @@ public class TestINodeAttributeProvider {
               ancestorAccess, parentAccess, access, subAccess, ignoreEmptyDir);
         }
         CALLED.add("checkPermission|" + ancestorAccess + "|" + parentAccess + "|" + access);
-        CALLED.add("checkPermission|" + path);
       }
 
       @Override
@@ -95,7 +93,6 @@ public class TestINodeAttributeProvider {
         CALLED.add("checkPermission|" + authzContext.getAncestorAccess()
             + "|" + authzContext.getParentAccess() + "|" + authzContext
             .getAccess());
-        CALLED.add("checkPermission|" + authzContext.getPath());
       }
     }
 
@@ -112,12 +109,7 @@ 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() {
@@ -493,111 +485,4 @@ 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