浏览代码

HDFS-13087. Snapshotted encryption zone information should be immutable. Contributed by LiXin Ge, Siyao Meng.

Signed-off-by: Wei-Chiu Chuang <weichiu@apache.org>
Co-authored-by: Siyao Meng <smeng@cloudera.com>
Wei-Chiu Chuang 6 年之前
父节点
当前提交
380289a167

+ 66 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java

@@ -27,6 +27,9 @@ import java.util.TreeMap;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
@@ -40,6 +43,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
@@ -94,6 +98,34 @@ public class EncryptionZoneManager {
     String getKeyName() {
     String getKeyName() {
       return keyName;
       return keyName;
     }
     }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof EncryptionZoneInt)) {
+        return false;
+      }
+
+      EncryptionZoneInt b = (EncryptionZoneInt)o;
+      return new EqualsBuilder()
+          .append(inodeId, b.getINodeId())
+          .append(suite, b.getSuite())
+          .append(version, b.getVersion())
+          .append(keyName, b.getKeyName())
+          .isEquals();
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().
+          append(inodeId).
+          append(suite).
+          append(version).
+          append(keyName).
+          toHashCode();
+    }
   }
   }
 
 
   private TreeMap<Long, EncryptionZoneInt> encryptionZones = null;
   private TreeMap<Long, EncryptionZoneInt> encryptionZones = null;
@@ -166,8 +198,8 @@ public class EncryptionZoneManager {
    * <p/>
    * <p/>
    * Called while holding the FSDirectory lock.
    * Called while holding the FSDirectory lock.
    */
    */
-  boolean isInAnEZ(INodesInPath iip)
-      throws UnresolvedLinkException, SnapshotAccessControlException {
+  boolean isInAnEZ(INodesInPath iip) throws UnresolvedLinkException,
+      SnapshotAccessControlException, IOException {
     assert dir.hasReadLock();
     assert dir.hasReadLock();
     return (getEncryptionZoneForPath(iip) != null);
     return (getEncryptionZoneForPath(iip) != null);
   }
   }
@@ -188,7 +220,7 @@ public class EncryptionZoneManager {
    * <p/>
    * <p/>
    * Called while holding the FSDirectory lock.
    * Called while holding the FSDirectory lock.
    */
    */
-  String getKeyName(final INodesInPath iip) {
+  String getKeyName(final INodesInPath iip) throws IOException {
     assert dir.hasReadLock();
     assert dir.hasReadLock();
     EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
     EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
     if (ezi == null) {
     if (ezi == null) {
@@ -203,19 +235,43 @@ public class EncryptionZoneManager {
    * <p/>
    * <p/>
    * Called while holding the FSDirectory lock.
    * Called while holding the FSDirectory lock.
    */
    */
-  private EncryptionZoneInt getEncryptionZoneForPath(INodesInPath iip) {
+  private EncryptionZoneInt getEncryptionZoneForPath(INodesInPath iip)
+      throws IOException {
     assert dir.hasReadLock();
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     Preconditions.checkNotNull(iip);
     if (!hasCreatedEncryptionZone()) {
     if (!hasCreatedEncryptionZone()) {
       return null;
       return null;
     }
     }
+
+    int snapshotID = iip.getPathSnapshotId();
     for (int i = iip.length() - 1; i >= 0; i--) {
     for (int i = iip.length() - 1; i >= 0; i--) {
       final INode inode = iip.getINode(i);
       final INode inode = iip.getINode(i);
-      if (inode != null) {
+      if (inode == null || !inode.isDirectory()) {
+        //not found or not a directory, encryption zone is supported on
+        //directory only.
+        continue;
+      }
+      if (snapshotID == Snapshot.CURRENT_STATE_ID) {
         final EncryptionZoneInt ezi = encryptionZones.get(inode.getId());
         final EncryptionZoneInt ezi = encryptionZones.get(inode.getId());
         if (ezi != null) {
         if (ezi != null) {
           return ezi;
           return ezi;
         }
         }
+      } else {
+        XAttr xAttr = FSDirXAttrOp.unprotectedGetXAttrByPrefixedName(
+            inode, snapshotID, CRYPTO_XATTR_ENCRYPTION_ZONE);
+        if (xAttr != null) {
+          try {
+            final HdfsProtos.ZoneEncryptionInfoProto ezProto =
+                HdfsProtos.ZoneEncryptionInfoProto.parseFrom(xAttr.getValue());
+            return new EncryptionZoneInt(
+                inode.getId(), PBHelperClient.convert(ezProto.getSuite()),
+                PBHelperClient.convert(ezProto.getCryptoProtocolVersion()),
+                ezProto.getKeyName());
+          } catch (InvalidProtocolBufferException e) {
+            throw new IOException("Could not parse encryption zone for inode "
+                + iip.getPath(), e);
+          }
+        }
       }
       }
     }
     }
     return null;
     return null;
@@ -228,7 +284,8 @@ public class EncryptionZoneManager {
    * <p/>
    * <p/>
    * Called while holding the FSDirectory lock.
    * Called while holding the FSDirectory lock.
    */
    */
-  private EncryptionZoneInt getParentEncryptionZoneForPath(INodesInPath iip) {
+  private EncryptionZoneInt getParentEncryptionZoneForPath(INodesInPath iip)
+      throws IOException {
     assert dir.hasReadLock();
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     Preconditions.checkNotNull(iip);
     INodesInPath parentIIP = iip.getParentINodesInPath();
     INodesInPath parentIIP = iip.getParentINodesInPath();
@@ -242,7 +299,8 @@ public class EncryptionZoneManager {
    * @param iip The INodesInPath of the path to check
    * @param iip The INodesInPath of the path to check
    * @return the EncryptionZone representing the ez for the path.
    * @return the EncryptionZone representing the ez for the path.
    */
    */
-  EncryptionZone getEZINodeForPath(INodesInPath iip) {
+  EncryptionZone getEZINodeForPath(INodesInPath iip)
+      throws IOException {
     final EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
     final EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
     if (ezi == null) {
     if (ezi == null) {
       return null;
       return null;
@@ -283,7 +341,7 @@ public class EncryptionZoneManager {
     }
     }
 
 
     if (srcInEZ) {
     if (srcInEZ) {
-      if (srcParentEZI != dstParentEZI) {
+      if (!srcParentEZI.equals(dstParentEZI)) {
         final String srcEZPath = getFullPathName(srcParentEZI);
         final String srcEZPath = getFullPathName(srcParentEZI);
         final String dstEZPath = getFullPathName(dstParentEZI);
         final String dstEZPath = getFullPathName(dstParentEZI);
         final StringBuilder sb = new StringBuilder(srcIIP.getPath());
         final StringBuilder sb = new StringBuilder(srcIIP.getPath());

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java

@@ -198,7 +198,7 @@ final class FSDirEncryptionZoneOp {
   }
   }
 
 
   static EncryptionZone getEZForPath(final FSDirectory fsd,
   static EncryptionZone getEZForPath(final FSDirectory fsd,
-      final INodesInPath iip) {
+      final INodesInPath iip) throws IOException {
     fsd.readLock();
     fsd.readLock();
     try {
     try {
       return fsd.ezManager.getEZINodeForPath(iip);
       return fsd.ezManager.getEZINodeForPath(iip);
@@ -282,7 +282,8 @@ final class FSDirEncryptionZoneOp {
       final CipherSuite suite = encryptionZone.getSuite();
       final CipherSuite suite = encryptionZone.getSuite();
       final String keyName = encryptionZone.getKeyName();
       final String keyName = encryptionZone.getKeyName();
       XAttr fileXAttr = FSDirXAttrOp.unprotectedGetXAttrByPrefixedName(
       XAttr fileXAttr = FSDirXAttrOp.unprotectedGetXAttrByPrefixedName(
-          iip, CRYPTO_XATTR_FILE_ENCRYPTION_INFO);
+          iip.getLastINode(), iip.getPathSnapshotId(),
+          CRYPTO_XATTR_FILE_ENCRYPTION_INFO);
 
 
       if (fileXAttr == null) {
       if (fileXAttr == null) {
         NameNode.LOG.warn("Could not find encryption XAttr for file " +
         NameNode.LOG.warn("Could not find encryption XAttr for file " +
@@ -316,7 +317,7 @@ final class FSDirEncryptionZoneOp {
    */
    */
   static FileEncryptionInfo getFileEncryptionInfo(FSDirectory dir,
   static FileEncryptionInfo getFileEncryptionInfo(FSDirectory dir,
       INodesInPath iip, EncryptionKeyInfo ezInfo)
       INodesInPath iip, EncryptionKeyInfo ezInfo)
-          throws RetryStartFileException {
+          throws RetryStartFileException, IOException {
     FileEncryptionInfo feInfo = null;
     FileEncryptionInfo feInfo = null;
     final EncryptionZone zone = getEZForPath(dir, iip);
     final EncryptionZone zone = getEZForPath(dir, iip);
     if (zone != null) {
     if (zone != null) {
@@ -339,7 +340,8 @@ final class FSDirEncryptionZoneOp {
   }
   }
 
 
   static boolean isInAnEZ(final FSDirectory fsd, final INodesInPath iip)
   static boolean isInAnEZ(final FSDirectory fsd, final INodesInPath iip)
-      throws UnresolvedLinkException, SnapshotAccessControlException {
+      throws UnresolvedLinkException, SnapshotAccessControlException,
+      IOException {
     if (!fsd.ezManager.hasCreatedEncryptionZone()) {
     if (!fsd.ezManager.hasCreatedEncryptionZone()) {
       return false;
       return false;
     }
     }

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java

@@ -360,16 +360,18 @@ class FSDirXAttrOp {
       String prefixedName) throws IOException {
       String prefixedName) throws IOException {
     fsd.readLock();
     fsd.readLock();
     try {
     try {
-      return XAttrStorage.readINodeXAttrByPrefixedName(iip, prefixedName);
+      return XAttrStorage.readINodeXAttrByPrefixedName(iip.getLastINode(),
+          iip.getPathSnapshotId(), prefixedName);
     } finally {
     } finally {
       fsd.readUnlock();
       fsd.readUnlock();
     }
     }
   }
   }
 
 
   static XAttr unprotectedGetXAttrByPrefixedName(
   static XAttr unprotectedGetXAttrByPrefixedName(
-      INodesInPath iip, String prefixedName)
+      INode inode, int snapshotId, String prefixedName)
       throws IOException {
       throws IOException {
-    return XAttrStorage.readINodeXAttrByPrefixedName(iip, prefixedName);
+    return XAttrStorage.readINodeXAttrByPrefixedName(
+        inode, snapshotId, prefixedName);
   }
   }
 
 
   private static void checkXAttrChangeAccess(
   private static void checkXAttrChangeAccess(

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

@@ -36,14 +36,13 @@ public class XAttrStorage {
    * <p/>
    * <p/>
    *
    *
    * @param inode INode to read
    * @param inode INode to read
-   * @param snapshotId
+   * @param snapshotId the snapshotId of the requested path
    * @param prefixedName xAttr name with prefix
    * @param prefixedName xAttr name with prefix
    * @return the xAttr
    * @return the xAttr
    */
    */
-  public static XAttr readINodeXAttrByPrefixedName(INodesInPath iip,
+  public static XAttr readINodeXAttrByPrefixedName(INode inode, int snapshotId,
       String prefixedName) {
       String prefixedName) {
-    XAttrFeature f =
-        iip.getLastINode().getXAttrFeature(iip.getPathSnapshotId());
+    XAttrFeature f = inode.getXAttrFeature(snapshotId);
     return f == null ? null : f.getXAttr(prefixedName);
     return f == null ? null : f.getXAttr(prefixedName);
   }
   }
 
 

+ 13 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -1424,11 +1424,20 @@ public class TestEncryptionZones {
     fsWrapper.mkdir(zone, FsPermission.getDirDefault(), true);
     fsWrapper.mkdir(zone, FsPermission.getDirDefault(), true);
     final Path snap2 = fs.createSnapshot(zoneParent, "snap2");
     final Path snap2 = fs.createSnapshot(zoneParent, "snap2");
     final Path snap2Zone = new Path(snap2, zone.getName());
     final Path snap2Zone = new Path(snap2, zone.getName());
+    assertEquals("Got unexpected ez path", zone.toString(),
+        dfsAdmin.getEncryptionZoneForPath(snap1Zone).getPath().toString());
     assertNull("Expected null ez path",
     assertNull("Expected null ez path",
         dfsAdmin.getEncryptionZoneForPath(snap2Zone));
         dfsAdmin.getEncryptionZoneForPath(snap2Zone));
 
 
-    // Create the encryption zone again
+    // Create the encryption zone again, and that shouldn't affect old snapshot
     dfsAdmin.createEncryptionZone(zone, TEST_KEY2, NO_TRASH);
     dfsAdmin.createEncryptionZone(zone, TEST_KEY2, NO_TRASH);
+    EncryptionZone ezSnap1 = dfsAdmin.getEncryptionZoneForPath(snap1Zone);
+    assertEquals("Got unexpected ez path", zone.toString(),
+        ezSnap1.getPath().toString());
+    assertEquals("Unexpected ez key", TEST_KEY, ezSnap1.getKeyName());
+    assertNull("Expected null ez path",
+        dfsAdmin.getEncryptionZoneForPath(snap2Zone));
+
     final Path snap3 = fs.createSnapshot(zoneParent, "snap3");
     final Path snap3 = fs.createSnapshot(zoneParent, "snap3");
     final Path snap3Zone = new Path(snap3, zone.getName());
     final Path snap3Zone = new Path(snap3, zone.getName());
     // Check that snap3's EZ has the correct settings
     // Check that snap3's EZ has the correct settings
@@ -1437,10 +1446,12 @@ public class TestEncryptionZones {
         ezSnap3.getPath().toString());
         ezSnap3.getPath().toString());
     assertEquals("Unexpected ez key", TEST_KEY2, ezSnap3.getKeyName());
     assertEquals("Unexpected ez key", TEST_KEY2, ezSnap3.getKeyName());
     // Check that older snapshots still have the old EZ settings
     // Check that older snapshots still have the old EZ settings
-    EncryptionZone ezSnap1 = dfsAdmin.getEncryptionZoneForPath(snap1Zone);
+    ezSnap1 = dfsAdmin.getEncryptionZoneForPath(snap1Zone);
     assertEquals("Got unexpected ez path", zone.toString(),
     assertEquals("Got unexpected ez path", zone.toString(),
         ezSnap1.getPath().toString());
         ezSnap1.getPath().toString());
     assertEquals("Unexpected ez key", TEST_KEY, ezSnap1.getKeyName());
     assertEquals("Unexpected ez key", TEST_KEY, ezSnap1.getKeyName());
+    assertNull("Expected null ez path",
+        dfsAdmin.getEncryptionZoneForPath(snap2Zone));
 
 
     // Check that listEZs only shows the current filesystem state
     // Check that listEZs only shows the current filesystem state
     ArrayList<EncryptionZone> listZones = Lists.newArrayList();
     ArrayList<EncryptionZone> listZones = Lists.newArrayList();