فهرست منبع

HDFS-6546. Add non-superuser capability to get the encryption zone for a specific path. (clamb)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1618022 13f79535-47bb-0310-9956-ffa450edef68
Charles Lamb 11 سال پیش
والد
کامیت
20dcb841ce
14فایلهای تغییر یافته به همراه258 افزوده شده و 3 حذف شده
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES-fs-encryption.txt
  2. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  3. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  4. 15 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  5. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  6. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  7. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  8. 20 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
  9. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  10. 35 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  11. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  12. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  13. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto
  14. 99 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-fs-encryption.txt

@@ -79,6 +79,9 @@ fs-encryption (Unreleased)
     HDFS-6834. Improve the configuration guidance in DFSClient when there 
     are no Codec classes found in configs. (umamahesh)
 
+    HDFS-6546. Add non-superuser capability to get the encryption zone
+    for a specific path. (clamb)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -154,6 +154,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
+import org.apache.hadoop.hdfs.protocol.EncryptionZoneWithId;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -2880,6 +2881,18 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  public EncryptionZone getEZForPath(String src)
+          throws IOException {
+    checkOpen();
+    try {
+      final EncryptionZoneWithId ezi = namenode.getEZForPath(src);
+      return (ezi.getId() < 0) ? null : ezi;
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     UnresolvedPathException.class);
+    }
+  }
+
   public RemoteIterator<EncryptionZone> listEncryptionZones()
       throws IOException {
     checkOpen();

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -1805,6 +1805,13 @@ public class DistributedFileSystem extends FileSystem {
     dfs.createEncryptionZone(getPathName(path), keyName);
   }
 
+  /* HDFS only */
+  public EncryptionZone getEZForPath(Path path)
+          throws IOException {
+    Preconditions.checkNotNull(path);
+    return dfs.getEZForPath(getPathName(path));
+  }
+
   /* HDFS only */
   public RemoteIterator<EncryptionZone> listEncryptionZones()
       throws IOException {

+ 15 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -21,7 +21,6 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.EnumSet;
-import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -247,6 +246,21 @@ public class HdfsAdmin {
     dfs.createEncryptionZone(path, keyName);
   }
 
+  /**
+   * Get the path of the encryption zone for a given file or directory.
+   *
+   * @param path The path to get the ez for.
+   *
+   * @return The EncryptionZone of the ez, or null if path is not in an ez.
+   * @throws IOException            if there was a general IO exception
+   * @throws AccessControlException if the caller does not have access to path
+   * @throws FileNotFoundException  if the path does not exist
+   */
+  public EncryptionZone getEncryptionZoneForPath(Path path)
+    throws IOException, AccessControlException, FileNotFoundException {
+    return dfs.getEZForPath(path);
+  }
+
   /**
    * Returns a RemoteIterator which can be used to list the encryption zones
    * in HDFS. For large numbers of encryption zones, the iterator will fetch

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -1275,6 +1275,13 @@ public interface ClientProtocol {
   public void createEncryptionZone(String src, String keyName)
     throws IOException;
 
+  /**
+   * Get the encryption zone for a path.
+   */
+  @Idempotent
+  public EncryptionZoneWithId getEZForPath(String src)
+    throws IOException;
+
   /**
    * Used to implement cursor-based batched listing of {@EncryptionZone}s.
    *

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -179,6 +179,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckA
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
@@ -1318,6 +1320,21 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     }
   }
 
+  @Override
+  public GetEZForPathResponseProto getEZForPath(
+      RpcController controller, GetEZForPathRequestProto req)
+      throws ServiceException {
+    try {
+      GetEZForPathResponseProto.Builder builder =
+          GetEZForPathResponseProto.newBuilder();
+      final EncryptionZoneWithId ret = server.getEZForPath(req.getSrc());
+      builder.setZone(PBHelper.convert(ret));
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
   @Override
   public ListEncryptionZonesResponseProto listEncryptionZones(
     RpcController controller, ListEncryptionZonesRequestProto req)

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -151,6 +151,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
@@ -1325,6 +1326,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
+  @Override
+  public EncryptionZoneWithId getEZForPath(String src)
+      throws IOException {
+    final GetEZForPathRequestProto.Builder builder =
+        GetEZForPathRequestProto.newBuilder();
+    builder.setSrc(src);
+    final GetEZForPathRequestProto req = builder.build();
+    try {
+      final EncryptionZonesProtos.GetEZForPathResponseProto response =
+          rpcProxy.getEZForPath(null, req);
+      return PBHelper.convert(response.getZone());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   @Override
   public BatchedEntries<EncryptionZoneWithId> listEncryptionZones(long id)
       throws IOException {

+ 20 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java

@@ -36,6 +36,9 @@ public class EncryptionZoneManager {
   public static Logger LOG = LoggerFactory.getLogger(EncryptionZoneManager
       .class);
 
+  private static final EncryptionZoneWithId NULL_EZ =
+      new EncryptionZoneWithId("", "", -1);
+
   /**
    * EncryptionZoneInt is the internal representation of an encryption zone. The
    * external representation of an EZ is embodied in an EncryptionZone and
@@ -57,7 +60,6 @@ public class EncryptionZoneManager {
     long getINodeId() {
       return inodeId;
     }
-
   }
 
   private final TreeMap<Long, EncryptionZoneInt> encryptionZones;
@@ -164,6 +166,23 @@ public class EncryptionZoneManager {
     return null;
   }
 
+  /**
+   * Returns an EncryptionZoneWithId representing the ez for a given path.
+   * Returns an empty marker EncryptionZoneWithId if path is not in an ez.
+   *
+   * @param iip The INodesInPath of the path to check
+   * @return the EncryptionZoneWithId representing the ez for the path.
+   */
+  EncryptionZoneWithId getEZINodeForPath(INodesInPath iip) {
+    final EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
+    if (ezi == null) {
+      return NULL_EZ;
+    } else {
+      return new EncryptionZoneWithId(getFullPathName(ezi), ezi.getKeyName(),
+          ezi.getINodeId());
+    }
+  }
+
   /**
    * Throws an exception if the provided path cannot be renamed into the
    * destination because of differing encryption zones.

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -2640,6 +2640,15 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  EncryptionZoneWithId getEZForPath(INodesInPath iip) {
+    readLock();
+    try {
+      return ezManager.getEZINodeForPath(iip);
+    } finally {
+      readUnlock();
+    }
+  }
+
   BatchedListEntries<EncryptionZoneWithId> listEncryptionZones(long prevId)
       throws IOException {
     readLock();

+ 35 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -8571,6 +8571,41 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     logAuditEvent(true, "createEncryptionZone", srcArg, null, resultingStat);
   }
 
+  /**
+   * Get the encryption zone for the specified path.
+   *
+   * @param srcArg the path of a file or directory to get the EZ for.
+   * @return the EZ of the of the path or null if none.
+   * @throws AccessControlException  if the caller is not the superuser.
+   * @throws UnresolvedLinkException if the path can't be resolved.
+   */
+  EncryptionZoneWithId getEZForPath(final String srcArg)
+    throws AccessControlException, UnresolvedLinkException, IOException {
+    String src = srcArg;
+    HdfsFileStatus resultingStat = null;
+    final byte[][] pathComponents =
+        FSDirectory.getPathComponentsForReservedPath(src);
+    boolean success = false;
+    final FSPermissionChecker pc = getPermissionChecker();
+    checkOperation(OperationCategory.READ);
+    readLock();
+    try {
+      if (isPermissionEnabled) {
+        checkPathAccess(pc, src, FsAction.READ);
+      }
+      checkOperation(OperationCategory.READ);
+      src = resolvePath(src, pathComponents);
+      final INodesInPath iip = dir.getINodesInPath(src, true);
+      final EncryptionZoneWithId ret = dir.getEZForPath(iip);
+      resultingStat = getAuditFileInfo(src, false);
+      success = true;
+      return ret;
+    } finally {
+      readUnlock();
+      logAuditEvent(success, "getEZForPath", srcArg, null, resultingStat);
+    }
+  }
+
   BatchedListEntries<EncryptionZoneWithId> listEncryptionZones(long prevId)
       throws IOException {
     boolean success = false;

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -1432,6 +1432,12 @@ class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.createEncryptionZone(src, keyName);
   }
 
+  @Override
+  public EncryptionZoneWithId getEZForPath(String src)
+    throws IOException {
+    return namesystem.getEZForPath(src);
+  }
+
   @Override
   public BatchedEntries<EncryptionZoneWithId> listEncryptionZones(
       long prevId) throws IOException {

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

@@ -799,4 +799,6 @@ service ClientNamenodeProtocol {
       returns(CreateEncryptionZoneResponseProto);
   rpc listEncryptionZones(ListEncryptionZonesRequestProto)
       returns(ListEncryptionZonesResponseProto);
+  rpc getEZForPath(GetEZForPathRequestProto)
+      returns(GetEZForPathResponseProto);
 }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto

@@ -55,3 +55,11 @@ message ListEncryptionZonesResponseProto {
   repeated EncryptionZoneWithIdProto zones = 1;
   required bool hasMore = 2;
 }
+
+message GetEZForPathRequestProto {
+    required string src = 1;
+}
+
+message GetEZForPathResponseProto {
+    required EncryptionZoneWithIdProto zone = 1;
+}

+ 99 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs;
 
 import java.io.File;
 import java.io.IOException;
-import java.security.NoSuchAlgorithmException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.List;
@@ -64,6 +63,7 @@ import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -337,6 +337,104 @@ public class TestEncryptionZones {
     });
   }
 
+  /**
+   * Test getEncryptionZoneForPath as a non super user.
+   */
+  @Test(timeout = 60000)
+  public void testGetEZAsNonSuperUser() throws Exception {
+
+    final UserGroupInformation user = UserGroupInformation.
+            createUserForTesting("user", new String[] { "mygroup" });
+
+    final Path testRoot = new Path(fsHelper.getTestRootDir());
+    final Path superPath = new Path(testRoot, "superuseronly");
+    final Path superPathFile = new Path(superPath, "file1");
+    final Path allPath = new Path(testRoot, "accessall");
+    final Path allPathFile = new Path(allPath, "file1");
+    final Path nonEZDir = new Path(testRoot, "nonEZDir");
+    final Path nonEZFile = new Path(nonEZDir, "file1");
+    final int len = 8192;
+
+    fsWrapper.mkdir(testRoot, new FsPermission((short) 0777), true);
+    fsWrapper.mkdir(superPath, new FsPermission((short) 0700), false);
+    fsWrapper.mkdir(allPath, new FsPermission((short) 0777), false);
+    fsWrapper.mkdir(nonEZDir, new FsPermission((short) 0777), false);
+    dfsAdmin.createEncryptionZone(superPath, TEST_KEY);
+    dfsAdmin.createEncryptionZone(allPath, TEST_KEY);
+    dfsAdmin.allowSnapshot(new Path("/"));
+    final Path newSnap = fs.createSnapshot(new Path("/"));
+    DFSTestUtil.createFile(fs, superPathFile, len, (short) 1, 0xFEED);
+    DFSTestUtil.createFile(fs, allPathFile, len, (short) 1, 0xFEED);
+    DFSTestUtil.createFile(fs, nonEZFile, len, (short) 1, 0xFEED);
+
+    user.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        final HdfsAdmin userAdmin =
+            new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+
+        // Check null arg
+        try {
+          userAdmin.getEncryptionZoneForPath(null);
+          fail("should have thrown NPE");
+        } catch (NullPointerException e) {
+          /*
+           * IWBNI we could use assertExceptionContains, but the NPE that is
+           * thrown has no message text.
+           */
+        }
+
+        // Check operation with accessible paths
+        assertEquals("expected ez path", allPath.toString(),
+            userAdmin.getEncryptionZoneForPath(allPath).getPath().
+            toString());
+        assertEquals("expected ez path", allPath.toString(),
+            userAdmin.getEncryptionZoneForPath(allPathFile).getPath().
+            toString());
+
+        // Check operation with inaccessible (lack of permissions) path
+        try {
+          userAdmin.getEncryptionZoneForPath(superPathFile);
+          fail("expected AccessControlException");
+        } catch (AccessControlException e) {
+          assertExceptionContains("Permission denied:", e);
+        }
+
+        // Check operation with non-ez paths
+        assertNull("expected null for non-ez path",
+            userAdmin.getEncryptionZoneForPath(nonEZDir));
+        assertNull("expected null for non-ez path",
+            userAdmin.getEncryptionZoneForPath(nonEZFile));
+
+        // Check operation with snapshots
+        String snapshottedAllPath = newSnap.toString() + allPath.toString();
+        assertEquals("expected ez path", allPath.toString(),
+            userAdmin.getEncryptionZoneForPath(
+                new Path(snapshottedAllPath)).getPath().toString());
+
+        /*
+         * Delete the file from the non-snapshot and test that it is still ok
+         * in the ez.
+         */
+        fs.delete(allPathFile, false);
+        assertEquals("expected ez path", allPath.toString(),
+            userAdmin.getEncryptionZoneForPath(
+                new Path(snapshottedAllPath)).getPath().toString());
+
+        // Delete the ez and make sure ss's ez is still ok.
+        fs.delete(allPath, true);
+        assertEquals("expected ez path", allPath.toString(),
+            userAdmin.getEncryptionZoneForPath(
+                new Path(snapshottedAllPath)).getPath().toString());
+        assertNull("expected null for deleted file path",
+            userAdmin.getEncryptionZoneForPath(allPathFile));
+        assertNull("expected null for deleted directory path",
+            userAdmin.getEncryptionZoneForPath(allPath));
+        return null;
+      }
+    });
+  }
+
   /**
    * Test success of Rename EZ on a directory which is already an EZ.
    */