瀏覽代碼

HDDS-945. Fix generics warnings in delegation token.
Contributed by Ajay Kumar.

Anu Engineer 6 年之前
父節點
當前提交
8978466fca

+ 42 - 31
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java

@@ -47,12 +47,12 @@ import java.util.concurrent.ConcurrentHashMap;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
-    extends OzoneSecretManager<T> {
+public class OzoneDelegationTokenSecretManager
+    extends OzoneSecretManager<OzoneTokenIdentifier> {
 
   private static final Logger LOG = LoggerFactory
       .getLogger(OzoneDelegationTokenSecretManager.class);
-  private final Map<T, TokenInfo> currentTokens;
+  private final Map<OzoneTokenIdentifier, TokenInfo> currentTokens;
   private final OzoneSecretStore store;
   private Thread tokenRemoverThread;
   private final long tokenRemoverScanInterval;
@@ -85,8 +85,8 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
   }
 
   @Override
-  public T createIdentifier() {
-    return (T) T.newInstance();
+  public OzoneTokenIdentifier createIdentifier() {
+    return OzoneTokenIdentifier.newInstance();
   }
 
   /**
@@ -94,8 +94,9 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
    *
    * @return T
    */
-  public T createIdentifier(Text owner, Text renewer, Text realUser) {
-    return (T) T.newInstance(owner, renewer, realUser);
+  public OzoneTokenIdentifier createIdentifier(Text owner, Text renewer,
+      Text realUser) {
+    return OzoneTokenIdentifier.newInstance(owner, renewer, realUser);
   }
 
   /**
@@ -106,17 +107,20 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
    * @param realUser
    * @return Token
    * @throws IOException to allow future exceptions to be added without breaking
-   *                     compatibility
+   * compatibility
    */
-  public Token<T> createToken(Text owner, Text renewer, Text realUser)
+  public Token<OzoneTokenIdentifier> createToken(Text owner, Text renewer,
+      Text realUser)
       throws IOException {
-    T identifier = createIdentifier(owner, renewer, realUser);
+    OzoneTokenIdentifier identifier = createIdentifier(owner, renewer,
+        realUser);
     updateIdentifierDetails(identifier);
 
     byte[] password = createPassword(identifier.getBytes(),
         getCurrentKey().getPrivateKey());
     addToTokenStore(identifier, password);
-    Token<T> token = new Token<>(identifier.getBytes(), password,
+    Token<OzoneTokenIdentifier> token = new Token<>(identifier.getBytes(),
+        password,
         identifier.getKind(), getService());
     if (LOG.isTraceEnabled()) {
       long expiryTime = identifier.getIssueDate() + getTokenRenewInterval();
@@ -134,7 +138,7 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
    * @param password
    * @throws IOException
    */
-  private void addToTokenStore(T identifier, byte[] password)
+  private void addToTokenStore(OzoneTokenIdentifier identifier, byte[] password)
       throws IOException {
     TokenInfo tokenInfo = new TokenInfo(identifier.getIssueDate()
         + getTokenRenewInterval(), password, identifier.getTrackingId());
@@ -147,7 +151,7 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
    *
    * @param identifier the identifier to validate
    */
-  private void updateIdentifierDetails(T identifier) {
+  private void updateIdentifierDetails(OzoneTokenIdentifier identifier) {
     int sequenceNum;
     long now = Time.monotonicNow();
     sequenceNum = incrementDelegationTokenSeqNum();
@@ -163,16 +167,17 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
    * @param token the token to renew
    * @param renewer the full principal name of the user doing the renewal
    * @return the new expiration time
-   * @throws InvalidToken           if the token is invalid
+   * @throws InvalidToken if the token is invalid
    * @throws AccessControlException if the user can't renew token
    */
   @Override
-  public synchronized long renewToken(Token<T> token, String renewer)
+  public synchronized long renewToken(Token<OzoneTokenIdentifier> token,
+      String renewer)
       throws IOException {
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     DataInputStream in = new DataInputStream(buf);
-    T id = (T) T.readProtoBuf(in);
-    if(LOG.isDebugEnabled()) {
+    OzoneTokenIdentifier id = OzoneTokenIdentifier.readProtoBuf(in);
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Token renewal for identifier: {}, total currentTokens: {}",
           formatTokenId(id), currentTokens.size());
     }
@@ -219,11 +224,13 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
    * Cancel a token by removing it from store and cache.
    *
    * @return Identifier of the canceled token
-   * @throws InvalidToken           for invalid token
+   * @throws InvalidToken for invalid token
    * @throws AccessControlException if the user isn't allowed to cancel
    */
-  public T cancelToken(Token<T> token, String canceller) throws IOException {
-    T id = (T) T.readProtoBuf(token.getIdentifier());
+  public OzoneTokenIdentifier cancelToken(Token<OzoneTokenIdentifier> token,
+      String canceller) throws IOException {
+    OzoneTokenIdentifier id = OzoneTokenIdentifier.readProtoBuf(
+        token.getIdentifier());
     LOG.debug("Token cancellation requested for identifier: {}",
         formatTokenId(id));
 
@@ -254,7 +261,8 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
   }
 
   @Override
-  public byte[] retrievePassword(T identifier) throws InvalidToken {
+  public byte[] retrievePassword(OzoneTokenIdentifier identifier)
+      throws InvalidToken {
     return validateToken(identifier).getPassword();
   }
 
@@ -262,7 +270,8 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
    * Checks if TokenInfo for the given identifier exists in database and if the
    * token is expired.
    */
-  public TokenInfo validateToken(T identifier) throws InvalidToken {
+  public TokenInfo validateToken(OzoneTokenIdentifier identifier)
+      throws InvalidToken {
     TokenInfo info = currentTokens.get(identifier);
     if (info == null) {
       throw new InvalidToken("token " + formatTokenId(identifier)
@@ -275,7 +284,7 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
           " expected renewal time: " + Time.formatTime(info.getRenewDate()));
     }
     if (!verifySignature(identifier, info.getPassword())) {
-      throw new InvalidToken("Tampared/Inavalid token.");
+      throw new InvalidToken("Tampered/Invalid token.");
     }
     return info;
   }
@@ -300,20 +309,21 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
     }
   }
 
-  private void loadTokenSecretState(OzoneManagerSecretState<T> state)
-      throws IOException {
+  private void loadTokenSecretState(
+      OzoneManagerSecretState<OzoneTokenIdentifier> state) throws IOException {
     LOG.info("Loading token state into token manager.");
     for (OzoneSecretKey key : state.ozoneManagerSecretState()) {
       allKeys.putIfAbsent(key.getKeyId(), key);
       incrementCurrentKeyId();
     }
-    for (Map.Entry<T, Long> entry : state.getTokenState().entrySet()) {
+    for (Map.Entry<OzoneTokenIdentifier, Long> entry :
+        state.getTokenState().entrySet()) {
       addPersistedDelegationToken(entry.getKey(), entry.getValue());
     }
   }
 
   private void addPersistedDelegationToken(
-      T identifier, long renewDate)
+      OzoneTokenIdentifier identifier, long renewDate)
       throws IOException {
     if (isRunning()) {
       // a safety check
@@ -397,13 +407,13 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
   /**
    * Remove expired delegation tokens from cache and persisted store.
    */
-  private void removeExpiredToken()  {
+  private void removeExpiredToken() {
     long now = Time.monotonicNow();
     synchronized (this) {
-      Iterator<Map.Entry<T,
+      Iterator<Map.Entry<OzoneTokenIdentifier,
           TokenInfo>> i = currentTokens.entrySet().iterator();
       while (i.hasNext()) {
-        Map.Entry<T,
+        Map.Entry<OzoneTokenIdentifier,
             TokenInfo> entry = i.next();
         long renewDate = entry.getValue().getRenewDate();
         if (renewDate < now) {
@@ -411,7 +421,7 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
           try {
             store.removeToken(entry.getKey());
           } catch (IOException e) {
-            if(LOG.isDebugEnabled()) {
+            if (LOG.isDebugEnabled()) {
               LOG.debug("Failed to remove expired token {}", entry.getValue());
             }
           }
@@ -421,6 +431,7 @@ public class OzoneDelegationTokenSecretManager<T extends OzoneTokenIdentifier>
   }
 
   private class ExpiredTokenRemover extends Thread {
+
     private long lastTokenCacheCleanup;
 
     @Override

+ 6 - 7
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneSecretStore.java

@@ -46,8 +46,7 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_CACHE_SIZE_MB;
 /**
  * SecretStore for Ozone Master.
  */
-public class OzoneSecretStore<T extends OzoneTokenIdentifier>
-    implements Closeable {
+public class OzoneSecretStore implements Closeable {
 
   private static final Logger LOG = LoggerFactory
       .getLogger(OzoneSecretStore.class);
@@ -142,7 +141,7 @@ public class OzoneSecretStore<T extends OzoneTokenIdentifier>
     }
   }
 
-  public void storeToken(T tokenId, Long renewDate)
+  public void storeToken(OzoneTokenIdentifier tokenId, Long renewDate)
       throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Storing token " + tokenId.getSequenceNumber());
@@ -168,12 +167,12 @@ public class OzoneSecretStore<T extends OzoneTokenIdentifier>
     }
   }
 
-  public void updateToken(T tokenId, Long renewDate)
+  public void updateToken(OzoneTokenIdentifier tokenId, Long renewDate)
       throws IOException {
     storeToken(tokenId, renewDate);
   }
 
-  public void removeToken(T tokenId)
+  public void removeToken(OzoneTokenIdentifier tokenId)
       throws IOException {
     byte[] dbKey = getTokenDBKey(tokenId);
     try {
@@ -228,7 +227,7 @@ public class OzoneSecretStore<T extends OzoneTokenIdentifier>
       throws IOException {
     long renewDate;
     DataInputStream in = new DataInputStream(new ByteArrayInputStream(data));
-    T tokenId = (T) T.readProtoBuf(in);
+    OzoneTokenIdentifier tokenId = OzoneTokenIdentifier.readProtoBuf(in);
     try {
       tokenId.readFields(in);
       renewDate = in.readLong();
@@ -243,7 +242,7 @@ public class OzoneSecretStore<T extends OzoneTokenIdentifier>
         TOKEN_MASTER_KEY_KEY_PREFIX + masterKey.getKeyId());
   }
 
-  private byte[] getTokenDBKey(T tokenId) {
+  private byte[] getTokenDBKey(OzoneTokenIdentifier tokenId) {
     return DFSUtil.string2Bytes(
         TOKEN_STATE_KEY_PREFIX + tokenId.getSequenceNumber());
   }

+ 3 - 4
hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/security/TestOzoneDelegationTokenSecretManager.java

@@ -44,8 +44,7 @@ import java.security.Signature;
  */
 public class TestOzoneDelegationTokenSecretManager {
 
-  private OzoneDelegationTokenSecretManager<OzoneTokenIdentifier>
-      secretManager;
+  private OzoneDelegationTokenSecretManager secretManager;
   private SecurityConfig securityConfig;
   private KeyPair keyPair;
   private long expiryTime;
@@ -209,10 +208,10 @@ public class TestOzoneDelegationTokenSecretManager {
   /**
    * Create instance of {@link OzoneDelegationTokenSecretManager}.
    */
-  private OzoneDelegationTokenSecretManager<OzoneTokenIdentifier>
+  private OzoneDelegationTokenSecretManager
       createSecretManager(OzoneConfiguration config, long tokenMaxLife,
       long expiry, long tokenRemoverScanTime) throws IOException {
-    return new OzoneDelegationTokenSecretManager<>(config, tokenMaxLife,
+    return new OzoneDelegationTokenSecretManager(config, tokenMaxLife,
         expiry, tokenRemoverScanTime, serviceRpcAdd);
   }
 }

+ 1 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java

@@ -55,7 +55,7 @@ public class TestOzoneRpcClient extends TestOzoneRpcClientAbstract {
   public static void shutdown() throws IOException {
     shutdownCluster();
   }
-  
+
   @Test
   public void testGetS3Secret() throws IOException {
     //Creates a secret since it does not exist

+ 3 - 4
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java

@@ -183,8 +183,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
           + StartupOption.HELP.getName() + " ]\n";
   private static final String OM_DAEMON = "om";
   private static boolean securityEnabled = false;
-  private static OzoneDelegationTokenSecretManager<OzoneTokenIdentifier>
-      delegationTokenMgr;
+  private static OzoneDelegationTokenSecretManager delegationTokenMgr;
   private OzoneBlockTokenSecretManager blockTokenMgr;
   private KeyPair keyPair;
   private CertificateClient certClient;
@@ -397,8 +396,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
         conf.getTimeDuration(OMConfigKeys.DELEGATION_TOKEN_RENEW_INTERVAL_KEY,
             OMConfigKeys.DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT,
             TimeUnit.MILLISECONDS);
-    Text omRpcAddressTxt = new Text(OmUtils.getOmRpcAddress(configuration));
-    return new OzoneDelegationTokenSecretManager<>(conf, tokenMaxLifetime,
+
+    return new OzoneDelegationTokenSecretManager(conf, tokenMaxLifetime,
         tokenRenewInterval, tokenRemoverScanInterval, omRpcAddressTxt);
   }