Browse Source

HDFS-13193. Various Improvements for BlockTokenSecretManager. Contributed by BELUGA BEHR.

Inigo Goiri 7 years ago
parent
commit
225d877092

+ 36 - 35
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java

@@ -22,18 +22,17 @@ import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.security.SecureRandom;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.io.WritableUtils;
@@ -41,11 +40,12 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.Timer;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.util.Timer;
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Multiset;
 
 /**
  * BlockTokenSecretManager can be instantiated in 2 modes, master mode
@@ -153,8 +153,9 @@ public class BlockTokenSecretManager extends
 
   /** Initialize block keys */
   private synchronized void generateKeys() {
-    if (!isMaster)
+    if (!isMaster) {
       return;
+    }
     /*
      * Need to set estimated expiry dates for currentKey and nextKey so that if
      * NN crashes, DN can still expire those keys. NN will stop using the newly
@@ -179,10 +180,10 @@ public class BlockTokenSecretManager extends
 
   /** Export block keys, only to be used in master mode */
   public synchronized ExportedBlockKeys exportKeys() {
-    if (!isMaster)
+    if (!isMaster) {
       return null;
-    if (LOG.isDebugEnabled())
-      LOG.debug("Exporting access keys");
+    }
+    LOG.debug("Exporting access keys");
     return new ExportedBlockKeys(true, keyUpdateInterval, tokenLifetime,
         currentKey, allKeys.values().toArray(new BlockKey[0]));
   }
@@ -203,16 +204,17 @@ public class BlockTokenSecretManager extends
    */
   public synchronized void addKeys(ExportedBlockKeys exportedKeys)
       throws IOException {
-    if (isMaster || exportedKeys == null)
+    if (isMaster || exportedKeys == null) {
       return;
+    }
     LOG.info("Setting block keys");
     removeExpiredKeys();
     this.currentKey = exportedKeys.getCurrentKey();
     BlockKey[] receivedKeys = exportedKeys.getAllKeys();
     for (int i = 0; i < receivedKeys.length; i++) {
-      if (receivedKeys[i] == null)
-        continue;
-      this.allKeys.put(receivedKeys[i].getKeyId(), receivedKeys[i]);
+      if (receivedKeys[i] != null) {
+        this.allKeys.put(receivedKeys[i].getKeyId(), receivedKeys[i]);
+      }
     }
   }
 
@@ -231,8 +233,9 @@ public class BlockTokenSecretManager extends
    * Update block keys, only to be used in master mode
    */
   synchronized boolean updateKeys() throws IOException {
-    if (!isMaster)
+    if (!isMaster) {
       return false;
+    }
 
     LOG.info("Updating block keys");
     removeExpiredKeys();
@@ -283,10 +286,10 @@ public class BlockTokenSecretManager extends
       ExtendedBlock block, BlockTokenIdentifier.AccessMode mode,
       StorageType[] storageTypes, String[] storageIds) throws InvalidToken {
     checkAccess(id, userId, block, mode);
-    if (storageTypes != null && storageTypes.length > 0) {
+    if (ArrayUtils.isNotEmpty(storageTypes)) {
       checkAccess(id.getStorageTypes(), storageTypes, "StorageTypes");
     }
-    if (storageIds != null && storageIds.length > 0) {
+    if (ArrayUtils.isNotEmpty(storageIds)) {
       checkAccess(id.getStorageIds(), storageIds, "StorageIDs");
     }
   }
@@ -296,26 +299,26 @@ public class BlockTokenSecretManager extends
       throws InvalidToken {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Checking access for user=" + userId + ", block=" + block
-          + ", access mode=" + mode + " using " + id.toString());
+          + ", access mode=" + mode + " using " + id);
     }
     if (userId != null && !userId.equals(id.getUserId())) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " doesn't belong to user " + userId);
     }
     if (!id.getBlockPoolId().equals(block.getBlockPoolId())) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " doesn't apply to block " + block);
     }
     if (id.getBlockId() != block.getBlockId()) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " doesn't apply to block " + block);
     }
     if (isExpired(id.getExpiryDate())) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " is expired.");
     }
     if (!id.getAccessModes().contains(mode)) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " doesn't have " + mode + " permission");
     }
   }
@@ -329,26 +332,23 @@ public class BlockTokenSecretManager extends
    */
   public static <T> void checkAccess(T[] candidates, T[] requested, String msg)
       throws InvalidToken {
-    if (requested.length == 0) {
+    if (ArrayUtils.isEmpty(requested)) {
       throw new InvalidToken("The request has no " + msg + ". "
           + "This is probably a configuration error.");
     }
-    if (candidates.length == 0) {
+    if (ArrayUtils.isEmpty(candidates)) {
       return;
     }
 
-    List unseenCandidates = new ArrayList<T>();
-    unseenCandidates.addAll(Arrays.asList(candidates));
+    Multiset<T> c = HashMultiset.create(Arrays.asList(candidates));
+
     for (T req : requested) {
-      final int index = unseenCandidates.indexOf(req);
-      if (index == -1) {
+      if (!c.remove(req)) {
         throw new InvalidToken("Block token with " + msg + " "
             + Arrays.toString(candidates)
             + " not valid for access with " + msg + " "
             + Arrays.toString(requested));
       }
-      Collections.swap(unseenCandidates, index, unseenCandidates.size()-1);
-      unseenCandidates.remove(unseenCandidates.size()-1);
     }
   }
 
@@ -367,7 +367,7 @@ public class BlockTokenSecretManager extends
     }
     checkAccess(id, userId, block, mode, storageTypes, storageIds);
     if (!Arrays.equals(retrievePassword(id), token.getPassword())) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " doesn't have the correct token password");
     }
   }
@@ -416,12 +416,13 @@ public class BlockTokenSecretManager extends
     synchronized (this) {
       key = currentKey;
     }
-    if (key == null)
+    if (key == null) {
       throw new IllegalStateException("currentKey hasn't been initialized.");
+    }
     identifier.setExpiryDate(timer.now() + tokenLifetime);
     identifier.setKeyId(key.getKeyId());
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Generating block token for " + identifier.toString());
+      LOG.debug("Generating block token for " + identifier);
     }
     return createPassword(identifier.getBytes(), key.getKey());
   }
@@ -438,7 +439,7 @@ public class BlockTokenSecretManager extends
   public byte[] retrievePassword(BlockTokenIdentifier identifier)
       throws InvalidToken {
     if (isExpired(identifier.getExpiryDate())) {
-      throw new InvalidToken("Block token with " + identifier.toString()
+      throw new InvalidToken("Block token with " + identifier
           + " is expired.");
     }
     BlockKey key = null;
@@ -447,7 +448,7 @@ public class BlockTokenSecretManager extends
     }
     if (key == null) {
       throw new InvalidToken("Can't re-compute password for "
-          + identifier.toString() + ", since the required block key (keyID="
+          + identifier + ", since the required block key (keyID="
           + identifier.getKeyId() + ") doesn't exist.");
     }
     return createPassword(identifier.getBytes(), key.getKey());