Browse Source

HDFS-10683. Make class Token$PrivateToken private. Contributed by John Zhuge.

Wei-Chiu Chuang 8 years ago
parent
commit
c5ca216915

+ 2 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java

@@ -104,12 +104,8 @@ public class Credentials implements Writable {
       for (Map.Entry<Text, Token<? extends TokenIdentifier>> e :
       for (Map.Entry<Text, Token<? extends TokenIdentifier>> e :
           tokenMap.entrySet()) {
           tokenMap.entrySet()) {
         Token<? extends TokenIdentifier> token = e.getValue();
         Token<? extends TokenIdentifier> token = e.getValue();
-        if (token instanceof Token.PrivateToken &&
-            ((Token.PrivateToken) token).getPublicService().equals(alias)) {
-          Token<? extends TokenIdentifier> privateToken =
-              new Token.PrivateToken<>(t);
-          privateToken.setService(token.getService());
-          tokensToAdd.put(e.getKey(), privateToken);
+        if (token.isPrivateCloneOf(alias)) {
+          tokensToAdd.put(e.getKey(), t.privateClone(token.getService()));
         }
         }
       }
       }
       tokenMap.putAll(tokensToAdd);
       tokenMap.putAll(tokensToAdd);

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -1584,7 +1584,7 @@ public class UserGroupInformation {
       Credentials creds = new Credentials(getCredentialsInternal());
       Credentials creds = new Credentials(getCredentialsInternal());
       Iterator<Token<?>> iter = creds.getAllTokens().iterator();
       Iterator<Token<?>> iter = creds.getAllTokens().iterator();
       while (iter.hasNext()) {
       while (iter.hasNext()) {
-        if (iter.next() instanceof Token.PrivateToken) {
+        if (iter.next().isPrivate()) {
           iter.remove();
           iter.remove();
         }
         }
       }
       }

+ 52 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java

@@ -222,23 +222,67 @@ public class Token<T extends TokenIdentifier> implements Writable {
     service = newService;
     service = newService;
   }
   }
 
 
+  /**
+   * Whether this is a private token.
+   * @return false always for non-private tokens
+   */
+  public boolean isPrivate() {
+    return false;
+  }
+
+  /**
+   * Whether this is a private clone of a public token.
+   * @param thePublicService the public service name
+   * @return false always for non-private tokens
+   */
+  public boolean isPrivateCloneOf(Text thePublicService) {
+    return false;
+  }
+
+  /**
+   * Create a private clone of a public token.
+   * @param newService the new service name
+   * @return a private token
+   */
+  public Token<T> privateClone(Text newService) {
+    return new PrivateToken<>(this, newService);
+  }
+
   /**
   /**
    * Indicates whether the token is a clone.  Used by HA failover proxy
    * Indicates whether the token is a clone.  Used by HA failover proxy
    * to indicate a token should not be visible to the user via
    * to indicate a token should not be visible to the user via
    * UGI.getCredentials()
    * UGI.getCredentials()
    */
    */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public static class PrivateToken<T extends TokenIdentifier> extends Token<T> {
+  static class PrivateToken<T extends TokenIdentifier> extends Token<T> {
     final private Text publicService;
     final private Text publicService;
 
 
-    public PrivateToken(Token<T> token) {
-      super(token);
-      publicService = new Text(token.getService());
+    PrivateToken(Token<T> publicToken, Text newService) {
+      super(publicToken.identifier, publicToken.password, publicToken.kind,
+          newService);
+      assert !publicToken.isPrivate();
+      publicService = publicToken.service;
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Cloned private token " + this + " from " + publicToken);
+      }
     }
     }
 
 
-    public Text getPublicService() {
-      return publicService;
+    /**
+     * Whether this is a private token.
+     * @return true always for private tokens
+     */
+    @Override
+    public boolean isPrivate() {
+      return true;
+    }
+
+    /**
+     * Whether this is a private clone of a public token.
+     * @param thePublicService the public service name
+     * @return true when the public service is the same as specified
+     */
+    @Override
+    public boolean isPrivateCloneOf(Text thePublicService) {
+      return publicService.equals(thePublicService);
     }
     }
 
 
     @Override
     @Override

+ 4 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java

@@ -890,8 +890,10 @@ public class TestUserGroupInformation {
     ugi.addToken(new Text("regular-token"), token);
     ugi.addToken(new Text("regular-token"), token);
 
 
     // Now add cloned private token
     // Now add cloned private token
-    ugi.addToken(new Text("private-token"), new Token.PrivateToken<TestTokenIdentifier>(token));
-    ugi.addToken(new Text("private-token1"), new Token.PrivateToken<TestTokenIdentifier>(token));
+    Text service = new Text("private-token");
+    ugi.addToken(service, token.privateClone(service));
+    Text service1 = new Text("private-token1");
+    ugi.addToken(service1, token.privateClone(service1));
 
 
     // Ensure only non-private tokens are returned
     // Ensure only non-private tokens are returned
     Collection<Token<? extends TokenIdentifier>> tokens = ugi.getCredentials().getAllTokens();
     Collection<Token<? extends TokenIdentifier>> tokens = ugi.getCredentials().getAllTokens();

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java

@@ -29,6 +29,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
+import static org.apache.hadoop.security.SecurityUtil.buildTokenService;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
@@ -56,7 +57,6 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.ipc.StandbyException;
-import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 
 
@@ -281,8 +281,7 @@ public class HAUtil {
         // exposed to the user via UGI.getCredentials(), otherwise these
         // exposed to the user via UGI.getCredentials(), otherwise these
         // cloned tokens may be inadvertently propagated to jobs
         // cloned tokens may be inadvertently propagated to jobs
         Token<DelegationTokenIdentifier> specificToken =
         Token<DelegationTokenIdentifier> specificToken =
-            new Token.PrivateToken<DelegationTokenIdentifier>(haToken);
-        SecurityUtil.setTokenService(specificToken, singleNNAddr);
+            haToken.privateClone(buildTokenService(singleNNAddr));
         Text alias = new Text(
         Text alias = new Text(
             HAUtilClient.buildTokenServicePrefixForLogicalUri(
             HAUtilClient.buildTokenServicePrefixForLogicalUri(
                 HdfsConstants.HDFS_URI_SCHEME)
                 HdfsConstants.HDFS_URI_SCHEME)