Przeglądaj źródła

HDFS-6516. List of Encryption Zones should be based on inodes (clamb)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1607770 13f79535-47bb-0310-9956-ffa450edef68
Charles Lamb 11 lat temu
rodzic
commit
c386652828

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

@@ -35,6 +35,8 @@ fs-encryption (Unreleased)
 
     HDFS-6625. Remove the Delete Encryption Zone function (clamb)
 
+    HDFS-6516. List of Encryption Zones should be based on inodes (clamb)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 133 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -27,8 +27,10 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.List;
 import java.util.ListIterator;
+import java.util.Map;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import com.google.protobuf.InvalidProtocolBufferException;
@@ -124,6 +126,9 @@ public class FSDirectory implements Closeable {
   public final static String DOT_INODES_STRING = ".inodes";
   public final static byte[] DOT_INODES = 
       DFSUtil.string2Bytes(DOT_INODES_STRING);
+  private final XAttr KEYID_XATTR =
+      XAttrHelper.buildXAttr(CRYPTO_XATTR_ENCRYPTION_ZONE, null);
+
   INodeDirectory rootDir;
   FSImage fsImage;  
   private final FSNamesystem namesystem;
@@ -136,6 +141,35 @@ public class FSDirectory implements Closeable {
   private long yieldCount = 0; // keep track of lock yield count.
   private final int inodeXAttrsLimit; //inode xattrs max limit
 
+  /*
+   * EncryptionZoneInt is the internal representation of an encryption
+   * zone. The external representation of an EZ is embodied in an
+   * EncryptionZone and contains the EZ's pathname.
+   */
+  private class EncryptionZoneInt {
+    private final String keyId;
+    private final long inodeId;
+
+    EncryptionZoneInt(String keyId, long inodeId) {
+      this.keyId = keyId;
+      this.inodeId = inodeId;
+    }
+
+    String getKeyId() {
+      return keyId;
+    }
+
+    long getINodeId() {
+      return inodeId;
+    }
+
+    String getFullPathName() {
+      return getInode(inodeId).getFullPathName();
+    }
+  }
+
+  private final Map<Long, EncryptionZoneInt> encryptionZones;
+
   // lock to protect the directory and BlockMap
   private final ReentrantReadWriteLock dirLock;
 
@@ -221,6 +255,7 @@ public class FSDirectory implements Closeable {
         + " times");
     nameCache = new NameCache<ByteArray>(threshold);
     namesystem = ns;
+    encryptionZones = new HashMap<Long, EncryptionZoneInt>();
   }
     
   private FSNamesystem getFSNamesystem() {
@@ -545,7 +580,7 @@ public class FSDirectory implements Closeable {
       return false;
     }
     
-    checkEncryptionZoneMoveValidity(src, dst);
+    checkEncryptionZoneMoveValidity(srcIIP, dstIIP, src);
     // Ensure dst has quota to accommodate rename
     verifyFsLimitsForRename(srcIIP, dstIIP);
     verifyQuotaForRename(srcIIP.getINodes(), dstIIP.getINodes());
@@ -750,7 +785,7 @@ public class FSDirectory implements Closeable {
       throw new IOException(error);
     }
 
-    checkEncryptionZoneMoveValidity(src, dst);
+    checkEncryptionZoneMoveValidity(srcIIP, dstIIP, src);
     final INode dstInode = dstIIP.getLastINode();
     List<INodeDirectorySnapshottable> snapshottableDirs = 
         new ArrayList<INodeDirectorySnapshottable>();
@@ -974,14 +1009,36 @@ public class FSDirectory implements Closeable {
     throw new IOException("rename from " + src + " to " + dst + " failed.");
   }
   
-  private void checkEncryptionZoneMoveValidity(String src, String dst)
+  boolean isInAnEZ(INodesInPath iip)
+    throws UnresolvedLinkException, SnapshotAccessControlException {
+    readLock();
+    try {
+      return (getEncryptionZoneForPath(iip) != null);
+    } finally {
+      readUnlock();
+    }
+  }
+
+  private EncryptionZoneInt getEncryptionZoneForPath(INodesInPath iip) {
+    Preconditions.checkNotNull(iip);
+    final INode[] inodes = iip.getINodes();
+    for (int i = inodes.length -1; i >= 0; i--) {
+      final INode inode = inodes[i];
+      if (inode != null) {
+        final EncryptionZoneInt ezi = encryptionZones.get(inode.getId());
+        if (ezi != null) {
+          return ezi;
+        }
+      }
+    }
+    return null;
+  }
+
+  private void checkEncryptionZoneMoveValidity(INodesInPath srcIIP,
+    INodesInPath dstIIP, String src)
     throws IOException {
-    final EncryptionZone srcEZ =
-      getFSNamesystem().getEncryptionZoneForPath(src);
-    final EncryptionZone dstEZ =
-      getFSNamesystem().getEncryptionZoneForPath(dst);
-    final boolean srcInEZ = srcEZ != null;
-    final boolean dstInEZ = dstEZ != null;
+    final boolean srcInEZ = (getEncryptionZoneForPath(srcIIP) != null);
+    final boolean dstInEZ = (getEncryptionZoneForPath(dstIIP) != null);
     if (srcInEZ) {
       if (!dstInEZ) {
         throw new IOException(src + " can't be moved from an encryption zone.");
@@ -993,12 +1050,18 @@ public class FSDirectory implements Closeable {
     }
 
     if (srcInEZ || dstInEZ) {
-      if (!srcEZ.getPath().equals(dstEZ.getPath())) {
+      final EncryptionZoneInt srcEZI = getEncryptionZoneForPath(srcIIP);
+      final EncryptionZoneInt dstEZI = getEncryptionZoneForPath(dstIIP);
+      Preconditions.checkArgument(srcEZI != null, "couldn't find src EZ?");
+      Preconditions.checkArgument(dstEZI != null, "couldn't find dst EZ?");
+      if (srcEZI != dstEZI) {
+        final String srcEZPath = srcEZI.getFullPathName();
+        final String dstEZPath = dstEZI.getFullPathName();
         final StringBuilder sb = new StringBuilder(src);
         sb.append(" can't be moved from encryption zone ");
-        sb.append(srcEZ.getPath());
+        sb.append(srcEZPath);
         sb.append(" to encryption zone ");
-        sb.append(dstEZ.getPath());
+        sb.append(dstEZPath);
         sb.append(".");
         throw new IOException(sb.toString());
       }
@@ -2167,6 +2230,18 @@ public class FSDirectory implements Closeable {
   public final void addToInodeMap(INode inode) {
     if (inode instanceof INodeWithAdditionalFields) {
       inodeMap.put(inode);
+      final XAttrFeature xaf = inode.getXAttrFeature();
+      if (xaf != null) {
+        final List<XAttr> xattrs = xaf.getXAttrs();
+        for (XAttr xattr : xattrs) {
+          final String xaName = XAttrHelper.getPrefixName(xattr);
+          if (CRYPTO_XATTR_ENCRYPTION_ZONE.equals(xaName)) {
+            encryptionZones.put(inode.getId(),
+              new EncryptionZoneInt(new String(xattr.getValue()),
+                                    inode.getId()));
+          }
+        }
+      }
     }
   }
   
@@ -2178,6 +2253,7 @@ public class FSDirectory implements Closeable {
       for (INode inode : inodes) {
         if (inode != null && inode instanceof INodeWithAdditionalFields) {
           inodeMap.remove(inode);
+          encryptionZones.remove(inode.getId());
         }
       }
     }
@@ -2682,6 +2758,8 @@ public class FSDirectory implements Closeable {
       for (ListIterator<XAttr> it = toFilter.listIterator(); it.hasNext()
           ;) {
         XAttr filter = it.next();
+        Preconditions.checkArgument(!KEYID_XATTR.equalsIgnoreValue(filter),
+            "The encryption zone xattr should never be deleted.");
         if (a.equalsIgnoreValue(filter)) {
           add = false;
           it.remove();
@@ -2705,17 +2783,42 @@ public class FSDirectory implements Closeable {
         throw new IOException(
           "Attempt to create an encryption zone for a non-empty directory.");
       }
+
+      final INodesInPath srcIIP = getINodesInPath4Write(src, false);
+      final EncryptionZoneInt ezi = getEncryptionZoneForPath(srcIIP);
+      if (ezi != null) {
+        throw new IOException("Directory " + src +
+          " is already in an encryption zone. (" + ezi.getFullPathName() + ")");
+      }
+
       final XAttr keyIdXAttr =
         XAttrHelper.buildXAttr(CRYPTO_XATTR_ENCRYPTION_ZONE, keyId.getBytes());
-      List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
+      final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
       xattrs.add(keyIdXAttr);
-      unprotectedSetXAttrs(src, xattrs, EnumSet.of(XAttrSetFlag.CREATE));
+      final INode inode = unprotectedSetXAttrs(src, xattrs,
+        EnumSet.of(XAttrSetFlag.CREATE));
+      encryptionZones.put(inode.getId(),
+          new EncryptionZoneInt(keyId, inode.getId()));
       return keyIdXAttr;
     } finally {
       writeUnlock();
     }
   }
 
+  List<EncryptionZone> listEncryptionZones() throws IOException {
+    readLock();
+    try {
+      final List<EncryptionZone> ret =
+        Lists.newArrayListWithExpectedSize(encryptionZones.size());
+      for (EncryptionZoneInt ezi : encryptionZones.values()) {
+        ret.add(new EncryptionZone(ezi.getFullPathName(), ezi.getKeyId()));
+      }
+      return ret;
+    } finally {
+      readUnlock();
+    }
+  }
+
   /**
    * Set the FileEncryptionInfo for an INode.
    */
@@ -2782,7 +2885,7 @@ public class FSDirectory implements Closeable {
     }
   }
   
-  void unprotectedSetXAttrs(final String src, final List<XAttr> xAttrs,
+  INode unprotectedSetXAttrs(final String src, final List<XAttr> xAttrs,
       final EnumSet<XAttrSetFlag> flag)
       throws QuotaExceededException, IOException {
     assert hasWriteLock();
@@ -2791,7 +2894,22 @@ public class FSDirectory implements Closeable {
     int snapshotId = iip.getLatestSnapshotId();
     List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
     List<XAttr> newXAttrs = setINodeXAttrs(existingXAttrs, xAttrs, flag);
+
+    /*
+     * If we're adding the encryption zone xattr, then add src to the list
+     * of encryption zones.
+     */
+    for (XAttr xattr : newXAttrs) {
+      final String xaName = XAttrHelper.getPrefixName(xattr);
+      if (CRYPTO_XATTR_ENCRYPTION_ZONE.equals(xaName)) {
+        final EncryptionZoneInt ez =
+            new EncryptionZoneInt(new String(xattr.getValue()), inode.getId());
+        encryptionZones.put(inode.getId(), ez);
+      }
+    }
+
     XAttrStorage.updateINodeXAttrs(inode, newXAttrs, snapshotId);
+    return inode;
   }
 
   List<XAttr> setINodeXAttrs(final List<XAttr> existingXAttrs,

+ 7 - 38
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -270,7 +270,6 @@ import org.mortbay.util.ajax.JSON;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
-import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
@@ -528,7 +527,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private KeyProvider provider = null;
   private KeyProvider.Options providerOptions = null;
 
-  private final Map<String, EncryptionZone> encryptionZones;
   private final CryptoCodec codec;
 
   private volatile boolean imageLoaded = false;
@@ -855,7 +853,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         auditLoggers.get(0) instanceof DefaultAuditLogger;
       this.retryCache = ignoreRetryCache ? null : initRetryCache(conf);
       this.nnConf = new NNConf(conf);
-      this.encryptionZones = new HashMap<String, EncryptionZone>();
     } catch(IOException e) {
       LOG.error(getClass().getSimpleName() + " initialization failed.", e);
       close();
@@ -2308,11 +2305,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @return chosen CipherSuite, or null if file is not in an EncryptionZone
    * @throws IOException
    */
-  private CipherSuite chooseCipherSuite(String src, List<CipherSuite> 
-      cipherSuites) throws UnknownCipherSuiteException {
-    EncryptionZone zone = getEncryptionZoneForPath(src);
+  private CipherSuite chooseCipherSuite(INodesInPath srcIIP, List<CipherSuite>
+      cipherSuites)
+      throws UnknownCipherSuiteException, UnresolvedLinkException,
+        SnapshotAccessControlException {
     // Not in an EZ
-    if (zone == null) {
+    if (!dir.isInAnEZ(srcIIP)) {
       return null;
     }
     CipherSuite chosen = null;
@@ -2469,7 +2467,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     FileEncryptionInfo feInfo = null;
-    CipherSuite suite = chooseCipherSuite(src, cipherSuites);
+    CipherSuite suite = chooseCipherSuite(iip, cipherSuites);
     if (suite != null) {
       Preconditions.checkArgument(!suite.equals(CipherSuite.UNKNOWN), 
           "Chose an UNKNOWN CipherSuite!");
@@ -3651,11 +3649,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             FsAction.ALL, true, false);
       }
 
-      final EncryptionZone ez = getEncryptionZoneForPath(src);
-      if (ez != null) {
-        encryptionZones.remove(src);
-      }
-
       long mtime = now();
       // Unlink the target directory from directory tree
       long filesRemoved = dir.delete(src, collectedBlocks, removedINodes,
@@ -8365,17 +8358,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       checkNameNodeSafeMode("Cannot create encryption zone on " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
 
-      EncryptionZone ez = getEncryptionZoneForPath(src);
-      if (ez != null) {
-        throw new IOException("Directory " + src +
-          " is already in an encryption zone. (" + ez.getPath() + ")");
-      }
-
       final XAttr keyIdXAttr = dir.createEncryptionZone(src, keyId);
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
       xAttrs.add(keyIdXAttr);
       getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
-      encryptionZones.put(src, new EncryptionZone(src, keyId));
       resultingStat = getAuditFileInfo(src, false);
     } finally {
       writeUnlock();
@@ -8400,7 +8386,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   List<EncryptionZone> listEncryptionZones() throws IOException {
-
     boolean success = false;
     checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);
@@ -8408,8 +8393,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       checkSuperuserPrivilege();
       checkOperation(OperationCategory.READ);
-      final List<EncryptionZone> ret =
-          Lists.newArrayList(encryptionZones.values());
+      final List<EncryptionZone> ret = dir.listEncryptionZones();
       success = true;
       return ret;
     } finally {
@@ -8418,21 +8402,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
-  /** Lookup the encryption zone of a path. */
-  EncryptionZone getEncryptionZoneForPath(String src) {
-    assert hasReadLock();
-    final String[] components = INode.getPathNames(src);
-    for (int i = components.length; i > 0; i--) {
-      final List<String> l = Arrays.asList(Arrays.copyOfRange(components, 0, i));
-      String p = Joiner.on(Path.SEPARATOR).join(l);
-      final EncryptionZone ret = encryptionZones.get(p);
-      if (ret != null) {
-        return ret;
-      }
-    }
-    return null;
-  }
-
   /**
    * Set xattr for a file or directory.
    *