Browse Source

HDFS-6716. Update usage of KeyProviderCryptoExtension APIs on NameNode. (wang)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1612438 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang 10 years ago
parent
commit
7b466b3b70

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

@@ -49,6 +49,9 @@ fs-encryption (Unreleased)
     HDFS-6490. Fix the keyid format for generated keys in
     FSNamesystem.createEncryptionZone (clamb)
 
+    HDFS-6716. Update usage of KeyProviderCryptoExtension APIs on NameNode.
+    (wang)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -563,8 +563,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS = "dfs.trustedchannel.resolver.class";
   public static final String DFS_DATA_TRANSFER_PROTECTION_KEY = "dfs.data.transfer.protection";
   public static final String DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY = "dfs.data.transfer.saslproperties.resolver.class";
-  public static final String DFS_NAMENODE_KEY_VERSION_REFRESH_INTERVAL_MS_KEY = "dfs.namenode.key.version.refresh.interval.ms";
-  public static final int DFS_NAMENODE_KEY_VERSION_REFRESH_INTERVAL_MS_DEFAULT = 5*60*1000;
   
   // Journal-node related configs. These are read on the JN side.
   public static final String  DFS_JOURNALNODE_EDITS_DIR_KEY = "dfs.journalnode.edits.dir";

+ 14 - 172
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java

@@ -3,27 +3,16 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 import java.util.EnumSet;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
@@ -53,37 +42,16 @@ public class EncryptionZoneManager {
    * contains the EZ's pathname.
    */
   private class EncryptionZoneInt {
-    private final String keyId;
+    private final String keyName;
     private final long inodeId;
 
-    private final HashSet<KeyVersion> keyVersions;
-    private KeyVersion latestVersion;
-
-    EncryptionZoneInt(long inodeId, String keyId) {
-      this.keyId = keyId;
+    EncryptionZoneInt(long inodeId, String keyName) {
+      this.keyName = keyName;
       this.inodeId = inodeId;
-      keyVersions = Sets.newHashSet();
-      latestVersion = null;
-    }
-
-    KeyVersion getLatestKeyVersion() {
-      return latestVersion;
-    }
-
-    void addKeyVersion(KeyVersion version) {
-      Preconditions.checkNotNull(version);
-      if (!keyVersions.contains(version)) {
-        LOG.debug("Key {} has new key version {}", keyId, version);
-        keyVersions.add(version);
-      }
-      // Always set the latestVersion to not get stuck on an old version in
-      // racy situations. Should eventually converge thanks to the
-      // monitor.
-      latestVersion = version;
     }
 
-    String getKeyId() {
-      return keyId;
+    String getKeyName() {
+      return keyName;
     }
 
     long getINodeId() {
@@ -123,7 +91,6 @@ public class EncryptionZoneManager {
 
   private final Map<Long, EncryptionZoneInt> encryptionZones;
   private final FSDirectory dir;
-  private final ScheduledExecutorService monitor;
   private final KeyProvider provider;
 
   /**
@@ -131,118 +98,11 @@ public class EncryptionZoneManager {
    *
    * @param dir Enclosing FSDirectory
    */
-  public EncryptionZoneManager(FSDirectory dir, Configuration conf,
-      KeyProvider provider) {
-
+  public EncryptionZoneManager(FSDirectory dir, KeyProvider provider) {
     this.dir = dir;
     this.provider = provider;
     lock = new ReentrantReadWriteLock();
     encryptionZones = new HashMap<Long, EncryptionZoneInt>();
-
-    monitor = Executors.newScheduledThreadPool(1,
-        new ThreadFactoryBuilder()
-            .setDaemon(true)
-            .setNameFormat(EncryptionZoneMonitor.class.getSimpleName() + "-%d")
-            .build());
-    final int refreshMs = conf.getInt(
-        DFSConfigKeys.DFS_NAMENODE_KEY_VERSION_REFRESH_INTERVAL_MS_KEY,
-        DFSConfigKeys.DFS_NAMENODE_KEY_VERSION_REFRESH_INTERVAL_MS_DEFAULT
-    );
-    Preconditions.checkArgument(refreshMs >= 0, "%s cannot be negative",
-        DFSConfigKeys.DFS_NAMENODE_KEY_VERSION_REFRESH_INTERVAL_MS_KEY);
-    monitor.scheduleAtFixedRate(new EncryptionZoneMonitor(), 0, refreshMs,
-        TimeUnit.MILLISECONDS);
-  }
-
-  /**
-   * Periodically wakes up to fetch the latest version of each encryption
-   * zone key.
-   */
-  private class EncryptionZoneMonitor implements Runnable {
-    @Override
-    public void run() {
-      LOG.debug("Monitor waking up to refresh encryption zone key versions");
-      HashMap<Long, String> toFetch = Maps.newHashMap();
-      HashMap<Long, KeyVersion> toUpdate =
-          Maps.newHashMap();
-      // Determine the keyIds to fetch
-      readLock();
-      try {
-        for (EncryptionZoneInt ezi : encryptionZones.values()) {
-          toFetch.put(ezi.getINodeId(), ezi.getKeyId());
-        }
-      } finally {
-        readUnlock();
-      }
-      LOG.trace("Found {} keys to check", toFetch.size());
-      // Fetch the key versions while not holding the lock
-      for (Map.Entry<Long, String> entry : toFetch.entrySet()) {
-        try {
-          KeyVersion version = provider.getCurrentKey(entry.getValue());
-          toUpdate.put(entry.getKey(), version);
-        } catch (IOException e) {
-          LOG.warn("Error while getting the current key for {} {}",
-              entry.getValue(), e);
-        }
-      }
-      LOG.trace("Fetched {} key versions from KeyProvider", toUpdate.size());
-      // Update the key versions for each encryption zone
-      writeLock();
-      try {
-        for (Map.Entry<Long, KeyVersion> entry : toUpdate.entrySet()) {
-          EncryptionZoneInt ezi = encryptionZones.get(entry.getKey());
-          // zone might have been removed in the intervening time
-          if (ezi == null) {
-            continue;
-          }
-          ezi.addKeyVersion(entry.getValue());
-        }
-      } finally {
-        writeUnlock();
-      }
-    }
-  }
-
-  /**
-   * Forces the EncryptionZoneMonitor to run, waiting until completion.
-   */
-  @VisibleForTesting
-  public void kickMonitor() throws Exception {
-    Future future = monitor.submit(new EncryptionZoneMonitor());
-    future.get();
-  }
-
-  /**
-   * Immediately fetches the latest KeyVersion for an encryption zone,
-   * also updating the encryption zone.
-   *
-   * @param iip of the encryption zone
-   * @return latest KeyVersion
-   * @throws IOException on KeyProvider error
-   */
-  KeyVersion updateLatestKeyVersion(INodesInPath iip) throws IOException {
-    EncryptionZoneInt ezi;
-    readLock();
-    try {
-      ezi = getEncryptionZoneForPath(iip);
-    } finally {
-      readUnlock();
-    }
-    if (ezi == null) {
-      throw new IOException("Cannot update KeyVersion since iip is not within" +
-          " an encryption zone");
-    }
-
-    // Do not hold the lock while doing KeyProvider operations
-    KeyVersion version = provider.getCurrentKey(ezi.getKeyId());
-
-    writeLock();
-    try {
-      ezi.addKeyVersion(version);
-      return version;
-    } finally {
-      writeUnlock();
-    }
   }
 
   /**
@@ -305,37 +165,20 @@ public class EncryptionZoneManager {
     return dir.getInode(ezi.getINodeId()).getFullPathName();
   }
 
-  KeyVersion getLatestKeyVersion(final INodesInPath iip) {
-    readLock();
-    try {
-      EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
-      if (ezi == null) {
-        return null;
-      }
-      return ezi.getLatestKeyVersion();
-    } finally {
-      readUnlock();
-    }
-  }
-
   /**
-   * @return true if the provided <tt>keyVersionName</tt> is the name of a
-   * valid KeyVersion for the encryption zone of <tt>iip</tt>,
-   * and <tt>iip</tt> is within an encryption zone.
+   * Get the key name for an encryption zone. Returns null if <tt>iip</tt> is
+   * not within an encryption zone.
+   * <p/>
+   * Called while holding the FSDirectory lock.
    */
-  boolean isValidKeyVersion(final INodesInPath iip, String keyVersionName) {
+  String getKeyName(final INodesInPath iip) {
     readLock();
     try {
       EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
       if (ezi == null) {
-        return false;
-      }
-      for (KeyVersion ezVersion : ezi.keyVersions) {
-        if (keyVersionName.equals(ezVersion.getVersionName())) {
-          return true;
-        }
+        return null;
       }
-      return false;
+      return ezi.getKeyName();
     } finally {
       readUnlock();
     }
@@ -447,7 +290,6 @@ public class EncryptionZoneManager {
       dir.unprotectedSetXAttrs(src, xattrs, EnumSet.of(XAttrSetFlag.CREATE));
       // Re-get the new encryption zone add the latest key version
       ezi = getEncryptionZoneForPath(srcIIP);
-      ezi.addKeyVersion(keyVersion);
       return keyIdXAttr;
     } finally {
       writeUnlock();
@@ -466,7 +308,7 @@ public class EncryptionZoneManager {
       final List<EncryptionZone> ret =
           Lists.newArrayListWithExpectedSize(encryptionZones.size());
       for (EncryptionZoneInt ezi : encryptionZones.values()) {
-        ret.add(new EncryptionZone(getFullPathName(ezi), ezi.getKeyId()));
+        ret.add(new EncryptionZone(getFullPathName(ezi), ezi.getKeyName()));
       }
       return ret;
     } finally {

+ 3 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -227,7 +227,7 @@ public class FSDirectory implements Closeable {
     nameCache = new NameCache<ByteArray>(threshold);
     namesystem = ns;
 
-    ezManager = new EncryptionZoneManager(this, conf, ns.getProvider());
+    ezManager = new EncryptionZoneManager(this, ns.getProvider());
   }
     
   private FSNamesystem getFSNamesystem() {
@@ -2623,25 +2623,10 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  KeyVersion getLatestKeyVersion(INodesInPath iip) {
+  String getKeyName(INodesInPath iip) {
     readLock();
     try {
-      return ezManager.getLatestKeyVersion(iip);
-    } finally {
-      readUnlock();
-    }
-  }
-
-  KeyVersion updateLatestKeyVersion(INodesInPath iip) throws
-      IOException {
-    // No locking, this operation does not involve any FSDirectory operations
-    return ezManager.updateLatestKeyVersion(iip);
-  }
-
-  boolean isValidKeyVersion(INodesInPath iip, String keyVersionName) {
-    readLock();
-    try {
-      return ezManager.isValidKeyVersion(iip, keyVersionName);
+      return ezManager.getKeyName(iip);
     } finally {
       readUnlock();
     }

+ 62 - 66
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
+import static org.apache.hadoop.crypto.key.KeyProviderCryptoExtension
+    .EncryptedKeyVersion;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
@@ -2356,59 +2358,26 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   /**
-   * Create a new FileEncryptionInfo for a path. Also chooses an
-   * appropriate CipherSuite to use from the list provided by the
-   * client.
+   * Invoke KeyProvider APIs to generate an encrypted data encryption key for an
+   * encryption zone. Should not be called with any locks held.
    *
-   * @param src Target path
-   * @param pathComponents Target path split up into path components
-   * @param cipherSuites List of CipherSuites provided by the client
-   * @return a new FileEncryptionInfo, or null if path is not within an
-   * encryption
-   * zone.
+   * @param ezKeyName key name of an encryption zone
+   * @return New EDEK, or null if ezKeyName is null
    * @throws IOException
    */
-  private FileEncryptionInfo newFileEncryptionInfo(String src,
-      byte[][] pathComponents, List<CipherSuite> cipherSuites)
-      throws IOException {
-    INodesInPath iip = null;
-    CipherSuite suite = null;
-    KeyVersion latestEZKeyVersion = null;
-    readLock();
-    try {
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
-      iip = dir.getINodesInPath4Write(src);
-      // Nothing to do if the path is not within an EZ
-      if (!dir.isInAnEZ(iip)) {
-        return null;
-      }
-      suite = chooseCipherSuite(iip, cipherSuites);
-      if (suite != null) {
-        Preconditions.checkArgument(!suite.equals(CipherSuite.UNKNOWN),
-            "Chose an UNKNOWN CipherSuite!");
-      }
-      latestEZKeyVersion = dir.getLatestKeyVersion(iip);
-    } finally {
-      readUnlock();
-    }
-
-    // If the latest key version is null, need to fetch it and update
-    if (latestEZKeyVersion == null) {
-      latestEZKeyVersion = dir.updateLatestKeyVersion(iip);
+  private EncryptedKeyVersion generateEncryptedDataEncryptionKey(String
+      ezKeyName) throws IOException {
+    if (ezKeyName == null) {
+      return null;
     }
-    Preconditions.checkState(latestEZKeyVersion != null);
-
-    // Generate the EDEK while not holding the lock
-    KeyProviderCryptoExtension.EncryptedKeyVersion edek = null;
+    EncryptedKeyVersion edek = null;
     try {
-      edek = provider.generateEncryptedKey("");
+      edek = provider.generateEncryptedKey(ezKeyName);
     } catch (GeneralSecurityException e) {
       throw new IOException(e);
     }
     Preconditions.checkNotNull(edek);
-
-    return new FileEncryptionInfo(suite, edek.getEncryptedKey().getMaterial(),
-        edek.getIv(), edek.getKeyVersionName());
+    return edek;
   }
 
   /**
@@ -2490,11 +2459,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     waitForLoadingFSImage();
 
     /*
-     * We want to avoid holding any locks while creating a new
-     * FileEncryptionInfo, since this can be very slow. Since the path can
+     * We want to avoid holding any locks while doing KeyProvider operations,
+     * since they can be very slow. Since the path can
      * flip flop between being in an encryption zone and not in the meantime,
-     * we need to recheck the preconditions and generate a new
-     * FileEncryptionInfo in some circumstances.
+     * we need to recheck the preconditions and redo KeyProvider operations
+     * in some situations.
      *
      * A special RetryStartFileException is used to indicate that we should
      * retry creation of a FileEncryptionInfo.
@@ -2510,18 +2479,45 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         }
         shouldContinue = false;
         iters++;
-        // Optimistically generate a FileEncryptionInfo for this path.
-        FileEncryptionInfo feInfo =
-            newFileEncryptionInfo(src, pathComponents, cipherSuites);
 
-        // Try to create the file with this feInfo
+        // Optimistically determine CipherSuite and ezKeyName if the path is
+        // currently within an encryption zone
+        CipherSuite suite = null;
+        String ezKeyName = null;
+        readLock();
+        try {
+          src = FSDirectory.resolvePath(src, pathComponents, dir);
+          INodesInPath iip = dir.getINodesInPath4Write(src);
+          // Nothing to do if the path is not within an EZ
+          if (dir.isInAnEZ(iip)) {
+            suite = chooseCipherSuite(iip, cipherSuites);
+            if (suite != null) {
+              Preconditions.checkArgument(!suite.equals(CipherSuite.UNKNOWN),
+                  "Chose an UNKNOWN CipherSuite!");
+            }
+            ezKeyName = dir.getKeyName(iip);
+            Preconditions.checkState(ezKeyName != null);
+          }
+        } finally {
+          readUnlock();
+        }
+
+        Preconditions.checkState(
+            (suite == null && ezKeyName == null) ||
+            (suite != null && ezKeyName != null),
+            "Both suite and ezKeyName should both be null or not null");
+        // Generate EDEK if necessary while not holding the lock
+        EncryptedKeyVersion edek =
+            generateEncryptedDataEncryptionKey(ezKeyName);
+
+        // Try to create the file with the computed cipher suite and EDEK
         writeLock();
         try {
           checkOperation(OperationCategory.WRITE);
           checkNameNodeSafeMode("Cannot create file" + src);
           src = FSDirectory.resolvePath(src, pathComponents, dir);
           startFileInternal(pc, src, permissions, holder, clientMachine, create,
-              overwrite, createParent, replication, blockSize, feInfo,
+              overwrite, createParent, replication, blockSize, suite, edek,
               logRetryCache);
           stat = dir.getFileInfo(src, false);
         } catch (StandbyException se) {
@@ -2561,8 +2557,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private void startFileInternal(FSPermissionChecker pc, String src,
       PermissionStatus permissions, String holder, String clientMachine,
       boolean create, boolean overwrite, boolean createParent,
-      short replication, long blockSize, FileEncryptionInfo feInfo,
-      boolean logRetryEntry)
+      short replication, long blockSize, CipherSuite suite,
+      EncryptedKeyVersion edek, boolean logRetryEntry)
       throws FileAlreadyExistsException, AccessControlException,
       UnresolvedLinkException, FileNotFoundException,
       ParentNotDirectoryException, RetryStartFileException, IOException {
@@ -2575,21 +2571,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           " already exists as a directory");
     }
 
-    if (!dir.isInAnEZ(iip)) {
-      // If the path is not in an EZ, we don't need an feInfo.
-      // Null it out in case one was already generated.
-      feInfo = null;
-    } else {
-      // The path is now within an EZ, but no feInfo. Retry.
-      if (feInfo == null) {
+    FileEncryptionInfo feInfo = null;
+    if (dir.isInAnEZ(iip)) {
+      // The path is now within an EZ, but we're missing encryption parameters
+      if (suite == null || edek == null) {
         throw new RetryStartFileException();
       }
-      // It's in an EZ and we have a provided feInfo. Make sure the
-      // keyVersion of the encryption key used matches one of the keyVersions of
-      // the key of the encryption zone.
-      if (!dir.isValidKeyVersion(iip, feInfo.getEzKeyVersionName())) {
+      // Path is within an EZ and we have provided encryption parameters.
+      // Make sure that the generated EDEK matches the settings of the EZ.
+      String ezKeyName = dir.getKeyName(iip);
+      if (!ezKeyName.equals(edek.getKeyName())) {
         throw new RetryStartFileException();
       }
+      feInfo = new FileEncryptionInfo(suite, edek.getEncryptedKey()
+          .getMaterial(), edek.getIv(), edek.getKeyVersionName());
+      Preconditions.checkNotNull(feInfo);
     }
 
     final INodeFile myFile = INodeFile.valueOf(inode, src, true);

+ 0 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -2039,15 +2039,4 @@
   </description>
 </property>
 
-<property>
-  <name>dfs.namenode.key.version.refresh.interval.ms</name>
-  <value>300000</value>
-  <description>How frequently the namenode will attempt to fetch the latest
-      key version of encryption zone keys from the configured KeyProvider, in
-      milliseconds. New key versions are created when a key is rolled. This
-      setting thus controls the window of staleness where an old key version
-      is used after a key is rolled.
-  </description>
-</property>
-
 </configuration>

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

@@ -347,7 +347,6 @@ public class TestEncryptionZones {
     assertEquals("Expected 1 EZ", 1, zones.size());
     String keyId = zones.get(0).getKeyId();
     cluster.getNamesystem().getProvider().rollNewVersion(keyId);
-    cluster.getNamesystem().getFSDirectory().ezManager.kickMonitor();
     // Read them back in and compare byte-by-byte
     validateFiles(baseFile, encFile1, len);
     // Write a new enc file and validate