You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by om...@apache.org on 2011/03/04 04:49:48 UTC

svn commit: r1077186 - in /hadoop/common/branches/branch-0.20-security-patches: ./ src/core/org/apache/hadoop/security/token/delegation/ src/hdfs/org/apache/hadoop/hdfs/ src/hdfs/org/apache/hadoop/hdfs/protocol/ src/hdfs/org/apache/hadoop/hdfs/server/n...

Author: omalley
Date: Fri Mar  4 03:49:47 2011
New Revision: 1077186

URL: http://svn.apache.org/viewvc?rev=1077186&view=rev
Log:
commit b8fc25222b0f40a0c3e4b8c07f4d75673f2afbfb
Author: Owen O'Malley <om...@apache.org>
Date:   Fri Feb 19 15:58:46 2010 -0800

    HADOOP-6551, HDFS-986, MAPREDUCE-1503. Change API for tokens to throw
    exceptions instead of returning booleans. (omalley)
    
    +++ b/YAHOO-CHANGES.txt
    +    HADOOP-6551, HDFS-986, MAPREDUCE-1503. Change API for tokens to throw
    +    exceptions instead of returning booleans. (omalley)
    +
    +    HADOOP-6572. Makes sure that SASL encryption and push to responder queue
    +    for the RPC response happens atomically. (Kan Zhang via ddas)
    +    HADOOP-6559. Makes the RPC client automatically re-login when the SASL
    +    connection setup fails. This is applicable to only keytab based logins.
    +    (ddas)
    +    HADOOP-6552. Puts renewTGT=true and useTicketCache=true for the keytab
    +    kerberos options. (ddas)
    +    HADOOP-6547, HDFS-949, MAPREDUCE-1470. Move Delegation token into Common
    +    so that we can use it for MapReduce also. It is a combined patch for
    +    common, hdfs and mr. (jitendra)
    +    HADOOP-6510,HDFS-935,MAPREDUCE-1464. Support for doAs to allow
    +    authenticated superuser to impersonate proxy users. It is a combined
    +    patch with compatible fixes in HDFS and MR. (jitendra)
    +    MAPREDUCE-1457. Fixes JobTracker to get the FileSystem object within
    +    getStagingAreaDir within a privileged block. Fixes Child.java to use the
    +    appropriate UGIs while getting the TaskUmbilicalProtocol proxy and while
    +    executing the task. Contributed by Jakob Homan. (ddas)
    +    Distributed Cache and Distcp. Also, provides a config
    +    mapreduce.job.hdfs-servers that the jobs can populate with a comma
    +    separated list of namenodes. The job client automatically fetches
    +    delegation tokens from those namenodes.
    +    HADOOP-4656, HDFS-685, MAPREDUCE-1083. Use the user-to-groups mapping
    +    service in the NameNode and JobTracker. Combined patch for these 3 jiras
    +    otherwise tests fail. (Jitendra Nath Pandey)
    +    HDFS-781. Namenode metrics PendingDeletionBlocks is not decremented.
    +    (suresh)

Modified:
    hadoop/common/branches/branch-0.20-security-patches/.gitignore
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobClient.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/security/TestDelegationToken.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/token/delegation/TestDelegationToken.java

Modified: hadoop/common/branches/branch-0.20-security-patches/.gitignore
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/.gitignore?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/.gitignore (original)
+++ hadoop/common/branches/branch-0.20-security-patches/.gitignore Fri Mar  4 03:49:47 2011
@@ -19,6 +19,8 @@
 .settings
 .svn
 build/
+build.properties
+build-fi/
 conf/masters
 conf/slaves
 conf/hadoop-env.sh
@@ -29,6 +31,8 @@ conf/hdfs-site.xml
 conf/hadoop-policy.xml
 conf/capacity-scheduler.xml
 docs/api/
+ivy/ivy-*.jar
+ivy/maven-ant-tasks-*.jar
 logs/
 src/contrib/ec2/bin/hadoop-ec2-env.sh
 src/contrib/index/conf/index-config.xml

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java Fri Mar  4 03:49:47 2011
@@ -34,6 +34,8 @@ import javax.crypto.SecretKey;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.util.Daemon;
@@ -185,11 +187,15 @@ extends AbstractDelegationTokenIdentifie
   }
 
   /**
-   * Renew a delegation token. Canceled tokens are not renewed. Return true if
-   * the token is successfully renewed; false otherwise.
+   * Renew a delegation token.
+   * @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 AccessControlException if the user can't renew token
    */
-  public Boolean renewToken(Token<TokenIdent> token,
-      String renewer) throws InvalidToken, IOException {
+  public long renewToken(Token<TokenIdent> token,
+                         String renewer) throws InvalidToken, IOException {
     long now = System.currentTimeMillis();
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     DataInputStream in = new DataInputStream(buf);
@@ -197,71 +203,76 @@ extends AbstractDelegationTokenIdentifie
     id.readFields(in);
     synchronized (currentTokens) {
       if (currentTokens.get(id) == null) {
-        LOG.warn("Renewal request for unknown token");
-        return false;
+        throw new InvalidToken("Renewal request for unknown token");
       }
     }
     if (id.getMaxDate() < now) {
-      LOG.warn("Client " + renewer + " tries to renew an expired token");
-      return false;
+      throw new InvalidToken("User " + renewer + 
+                             " tried to renew an expired token");
     }
-    if (id.getRenewer() == null || !id.getRenewer().toString().equals(renewer)) {
-      LOG.warn("Client " + renewer + " tries to renew a token with "
-          + "renewer specified as " + id.getRenewer());
-      return false;
+    if (id.getRenewer() == null) {
+      throw new AccessControlException("User " + renewer + 
+                                       " tried to renew a token without " +
+                                       "a renewer");
+    }
+    if (!id.getRenewer().toString().equals(renewer)) {
+      throw new AccessControlException("Client " + renewer + 
+                                       " tries to renew a token with " +
+                                       "renewer specified as " + 
+                                       id.getRenewer());
     }
     DelegationKey key = null;
     synchronized (this) {
       key = allKeys.get(id.getMasterKeyId());
     }
     if (key == null) {
-      LOG.warn("Unable to find master key for keyId=" + id.getMasterKeyId() 
-          + " from cache. Failed to renew an unexpired token with sequenceNumber=" 
-          + id.getSequenceNumber() + ", issued by this key");
-      return false;
+      throw new InvalidToken("Unable to find master key for keyId=" + 
+                             id.getMasterKeyId() +
+                             " from cache. Failed to renew an unexpired token"+
+                             " with sequenceNumber=" + id.getSequenceNumber());
     }
     byte[] password = createPassword(token.getIdentifier(), key.getKey());
     if (!Arrays.equals(password, token.getPassword())) {
-      LOG.warn("Client " + renewer + " is trying to renew a token with wrong password");
-      return false;
+      throw new AccessControlException("Client " + renewer + 
+                                       " is trying to renew a token with " +
+                                       "wrong password");
     }
     DelegationTokenInformation info = new DelegationTokenInformation(
         Math.min(id.getMaxDate(), now + tokenRenewInterval), password);
     synchronized (currentTokens) {
       currentTokens.put(id, info);
     }
-    return true;
+    return info.getRenewDate();
   }
   
   /**
-   * Cancel a token by removing it from cache. Return true if 
-   * token exists in cache; false otherwise.
+   * Cancel a token by removing it from cache.
+   * @throws InvalidToken for invalid token
+   * @throws AccessControlException if the user isn't allowed to cancel
    */
-  public Boolean cancelToken(Token<TokenIdent> token,
+  public void cancelToken(Token<TokenIdent> token,
       String canceller) throws IOException {
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     DataInputStream in = new DataInputStream(buf);
     TokenIdent id = createIdentifier();
     id.readFields(in);
-    if (id.getRenewer() == null) {
-      LOG.warn("Renewer is null: Invalid Identifier");
-      return false;
-    }
     if (id.getUser() == null) {
-      LOG.warn("owner is null: Invalid Identifier");
-      return false;
+      throw new InvalidToken("Token with no owner");
     }
     String owner = id.getUser().getUserName();
-    String renewer = id.getRenewer().toString();
-    if (!canceller.equals(owner) && !canceller.equals(renewer)) {
-      LOG.warn(canceller + " is not authorized to cancel the token");
-      return false;
+    Text renewer = id.getRenewer();
+    if (!canceller.equals(owner) && 
+        (renewer == null || !canceller.equals(renewer.toString()))) {
+      throw new AccessControlException(canceller + 
+                                      " is not authorized to cancel the token");
     }
     DelegationTokenInformation info = null;
     synchronized (currentTokens) {
       info = currentTokens.remove(id);
     }
-    return info != null;
+    if (info == null) {
+      throw new InvalidToken("Token not found");
+    }
   }
   
   /**

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java Fri Mar  4 03:49:47 2011
@@ -262,18 +262,24 @@ public class DFSClient implements FSCons
     return namenode.getDelegationToken(renewer);
   }
 
-  public Boolean renewDelegationToken(Token<DelegationTokenIdentifier> token)
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
     try {
       return namenode.renewDelegationToken(token);
     } catch (RemoteException re) {
-      throw re.unwrapRemoteException(InvalidToken.class);
+      throw re.unwrapRemoteException(InvalidToken.class,
+                                     AccessControlException.class);
     }
   }
 
-  public Boolean cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    return namenode.cancelDelegationToken(token);
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws InvalidToken, IOException {
+    try {
+      namenode.cancelDelegationToken(token);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(InvalidToken.class,
+                                     AccessControlException.class);
+    }
   }
   
   /**

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java Fri Mar  4 03:49:47 2011
@@ -509,10 +509,10 @@ public class DistributedFileSystem exten
    * Renew an existing delegation token.
    * 
    * @param token delegation token obtained earlier
-   * @return True if renewed successfully else false
+   * @return the new expiration time
    * @throws IOException
    */
-  public Boolean renewDelegationToken(Token<DelegationTokenIdentifier> token)
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
     return dfs.renewDelegationToken(token);
   }
@@ -521,11 +521,10 @@ public class DistributedFileSystem exten
    * Cancel an existing delegation token.
    * 
    * @param token delegation token
-   * @return True if canceled successfully else false
    * @throws IOException
    */
-  public Boolean cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
-    return dfs.cancelDelegationToken(token);
+    dfs.cancelDelegationToken(token);
   }
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java Fri Mar  4 03:49:47 2011
@@ -497,19 +497,18 @@ public interface ClientProtocol extends 
    * Renew an existing delegation token.
    *
    * @param token delegation token obtained earlier
-   * @return True if renewed successfully else false
+   * @return the new expiration time
    * @throws IOException
    */
-  public Boolean renewDelegationToken(Token<DelegationTokenIdentifier> token)
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException;
 
   /**
    * Cancel an existing delegation token.
    *
    * @param token delegation token
-   * @return True if canceled successfully else false
    * @throws IOException
    */
-  public Boolean cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException;
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Fri Mar  4 03:49:47 2011
@@ -4934,15 +4934,15 @@ public class FSNamesystem implements FSC
     return new Token<DelegationTokenIdentifier>(dtId, dtSecretManager);
   }
 
-  public Boolean renewDelegationToken(Token<DelegationTokenIdentifier> token)
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
     String renewer = UserGroupInformation.getCurrentUser().getShortUserName();
     return dtSecretManager.renewToken(token, renewer);
   }
 
-  public Boolean cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
     String canceller = UserGroupInformation.getCurrentUser().getShortUserName();
-    return dtSecretManager.cancelToken(token, canceller);
+    dtSecretManager.cancelToken(token, canceller);
   }
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java Fri Mar  4 03:49:47 2011
@@ -360,14 +360,16 @@ public class NameNode implements ClientP
     return namesystem.getDelegationToken(renewer);
   }
 
-  public Boolean renewDelegationToken(Token<DelegationTokenIdentifier> token)
+  @Override
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
     return namesystem.renewDelegationToken(token);
   }
 
-  public Boolean cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+  @Override
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
-    return namesystem.cancelDelegationToken(token);
+    namesystem.cancelDelegationToken(token);
   }
   
   /** {@inheritDoc} */

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobClient.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobClient.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobClient.java Fri Mar  4 03:49:47 2011
@@ -69,6 +69,7 @@ import org.apache.hadoop.mapreduce.JobSu
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapreduce.split.JobSplitWriter;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
@@ -1803,29 +1804,34 @@ public class JobClient extends Configure
   /**
    * Renew a delegation token
    * @param token the token to renew
-   * @return true if the renewal went well
+   * @return the new expiration time
    * @throws InvalidToken
    * @throws IOException
    */
-  public boolean renewDelegationToken(Token<DelegationTokenIdentifier> token)
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
   throws InvalidToken, IOException, InterruptedException {
     try {
       return jobSubmitClient.renewDelegationToken(token);
     } catch (RemoteException re) {
-      throw re.unwrapRemoteException(InvalidToken.class);
+      throw re.unwrapRemoteException(InvalidToken.class,
+                                     AccessControlException.class);
     }
   }
 
   /**
    * Cancel a delegation token from the JobTracker
    * @param token the token to cancel
-   * @return true if everything went well
    * @throws IOException
    */
-  public boolean cancelDelegationToken(Token<DelegationTokenIdentifier> token
-                                       ) throws IOException, 
-                                                InterruptedException {
-    return jobSubmitClient.cancelDelegationToken(token);
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token
+                                    ) throws IOException, 
+                                             InterruptedException {
+    try {
+      jobSubmitClient.cancelDelegationToken(token);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(InvalidToken.class,
+                                     AccessControlException.class);
+    }
   }
  
   /**

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java Fri Mar  4 03:49:47 2011
@@ -262,21 +262,20 @@ interface JobSubmissionProtocol extends 
   /**
    * Renew an existing delegation token
    * @param token the token to renew
-   * @return true if the token was successfully renewed
+   * @return the new expiration time
    * @throws IOException
    * @throws InterruptedException
    */ 
-  public boolean renewDelegationToken(Token<DelegationTokenIdentifier> token
-                                      ) throws IOException,
-                                               InterruptedException;
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token
+                                   ) throws IOException,
+                                            InterruptedException;
 
   /**
    * Cancel a delegation token.
    * @param token the token to cancel
-   * @return true if the token was successfully canceled
    * @throws IOException
    * @throws InterruptedException
    */ 
-  public boolean cancelDelegationToken(Token<DelegationTokenIdentifier> token
-                                       ) throws IOException,InterruptedException;
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token
+                                    ) throws IOException,InterruptedException;
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java Fri Mar  4 03:49:47 2011
@@ -3810,11 +3810,11 @@ public class JobTracker implements MRCon
    * Discard a current delegation token.
    */ 
   @Override
-  public boolean cancelDelegationToken(Token<DelegationTokenIdentifier> token
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token
                                        ) throws IOException,
                                                 InterruptedException {
     String user = UserGroupInformation.getCurrentUser().getUserName();
-    return secretManager.cancelToken(token, user);
+    secretManager.cancelToken(token, user);
   }  
   /**
    * Get a new delegation token.
@@ -3837,7 +3837,7 @@ public class JobTracker implements MRCon
    * Renew a delegation token to extend its lifetime.
    */ 
   @Override
-  public boolean renewDelegationToken(Token<DelegationTokenIdentifier> token
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token
                                       ) throws IOException,
                                                InterruptedException {
     String user = UserGroupInformation.getCurrentUser().getUserName();

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java Fri Mar  4 03:49:47 2011
@@ -541,10 +541,9 @@ class LocalJobRunner implements JobSubmi
   }
   
   @Override
-  public boolean cancelDelegationToken(Token<DelegationTokenIdentifier> token
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token
                                        ) throws IOException,
                                                 InterruptedException {
-    return false;
   }  
   @Override
   public Token<DelegationTokenIdentifier> 
@@ -552,9 +551,9 @@ class LocalJobRunner implements JobSubmi
     return null;
   }  
   @Override
-  public boolean renewDelegationToken(Token<DelegationTokenIdentifier> token
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token
                                       ) throws IOException,InterruptedException{
-    return false;
+    return 0;
   }  
 
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java Fri Mar  4 03:49:47 2011
@@ -222,14 +222,13 @@ public class TestDFSClientRetries extend
       return null;
     }
 
-    public Boolean renewDelegationToken(Token<DelegationTokenIdentifier> token)
+    public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
         throws InvalidToken, IOException {
-      return false;
+      return 0;
     }
 
-    public Boolean cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+    public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
         throws IOException {
-      return false;
     }
   }
   

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/security/TestDelegationToken.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/security/TestDelegationToken.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/security/TestDelegationToken.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/security/TestDelegationToken.java Fri Mar  4 03:49:47 2011
@@ -33,6 +33,8 @@ import org.apache.hadoop.hdfs.DFSConfigK
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -79,8 +81,13 @@ public class TestDelegationToken {
     Token<DelegationTokenIdentifier> token = generateDelegationToken(
         "SomeUser", "JobTracker");
     // Fake renewer should not be able to renew
-	  Assert.assertFalse(dtSecretManager.renewToken(token, "FakeRenewer"));
-	  Assert.assertTrue(dtSecretManager.renewToken(token, "JobTracker"));
+    try {
+  	  dtSecretManager.renewToken(token, "FakeRenewer");
+  	  Assert.fail("should have failed");
+    } catch (AccessControlException ace) {
+      // PASS
+    }
+	  dtSecretManager.renewToken(token, "JobTracker");
     DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
     byte[] tokenId = token.getIdentifier();
     identifier.readFields(new DataInputStream(
@@ -96,10 +103,15 @@ public class TestDelegationToken {
 	  } catch (InvalidToken e) {
 	    //Success
 	  }
-	  Assert.assertTrue(dtSecretManager.renewToken(token, "JobTracker"));
+	  dtSecretManager.renewToken(token, "JobTracker");
 	  Log.info("Sleep beyond the max lifetime");
 	  Thread.sleep(5000);
-	  Assert.assertFalse(dtSecretManager.renewToken(token, "JobTracker"));
+	  try {
+  	  dtSecretManager.renewToken(token, "JobTracker");
+  	  Assert.fail("should have been expired");
+	  } catch (InvalidToken it) {
+	    // PASS
+	  }
   }
   
   @Test 
@@ -109,9 +121,19 @@ public class TestDelegationToken {
     Token<DelegationTokenIdentifier> token = generateDelegationToken(
         "SomeUser", "JobTracker");
     //Fake renewer should not be able to renew
-    Assert.assertFalse(dtSecretManager.cancelToken(token, "FakeCanceller"));
-    Assert.assertTrue(dtSecretManager.cancelToken(token, "JobTracker"));
-    Assert.assertFalse(dtSecretManager.renewToken(token, "JobTracker"));
+    try {
+      dtSecretManager.cancelToken(token, "FakeCanceller");
+      Assert.fail("should have failed");
+    } catch (AccessControlException ace) {
+      // PASS
+    }
+    dtSecretManager.cancelToken(token, "JobTracker");
+    try {
+      dtSecretManager.renewToken(token, "JobTracker");
+      Assert.fail("should have failed");
+    } catch (InvalidToken it) {
+      // PASS
+    }
   }
   
   @Test
@@ -126,6 +148,7 @@ public class TestDelegationToken {
              new ByteArrayInputStream(tokenId)));
     Log.info("A valid token should have non-null password, and should be renewed successfully");
     Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
-    Assert.assertTrue(dtSecretManager.renewToken(token, "JobTracker"));
-  } 
+    dtSecretManager.renewToken(token, "JobTracker");
+  }
+ 
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java Fri Mar  4 03:49:47 2011
@@ -28,8 +28,11 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -85,12 +88,27 @@ public class TestDelegationToken {
     System.out.println("max time: " + maxTime);
     assertTrue("createTime < current", createTime < currentTime);
     assertTrue("current < maxTime", currentTime < maxTime);
-    assertTrue("alice renew", client.renewDelegationToken(token));
-    assertTrue("alice renew", client.renewDelegationToken(token));
-    assertFalse("bob renew", bobClient.renewDelegationToken(token));
-    assertFalse("bob cancel", bobClient.cancelDelegationToken(token));
-    assertTrue("alice cancel", client.cancelDelegationToken(token));
-    assertFalse("second alice cancel", client.cancelDelegationToken(token));
+    client.renewDelegationToken(token);
+    client.renewDelegationToken(token);
+    try {
+      bobClient.renewDelegationToken(token);
+      Assert.fail("bob renew");
+    } catch (AccessControlException ace) {
+      // PASS
+    }
+    try {
+      bobClient.cancelDelegationToken(token);
+      Assert.fail("bob renew");
+    } catch (AccessControlException ace) {
+      // PASS
+    }
+    client.cancelDelegationToken(token);
+    try {
+      client.cancelDelegationToken(token);
+      Assert.fail("second alice cancel");
+    } catch (InvalidToken it) {
+      // PASS
+    }
   }
 }
 

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/token/delegation/TestDelegationToken.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/token/delegation/TestDelegationToken.java?rev=1077186&r1=1077185&r2=1077186&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/token/delegation/TestDelegationToken.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/token/delegation/TestDelegationToken.java Fri Mar  4 03:49:47 2011
@@ -23,26 +23,28 @@ import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 
 import junit.framework.Assert;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
-import org.mortbay.log.Log;
 
 import static org.junit.Assert.*;
 
 public class TestDelegationToken {
+  private static final Log LOG = LogFactory.getLog(TestDelegationToken.class);
   private static final Text KIND = new Text("MY KIND");
 
   public static class TestDelegationTokenIdentifier 
@@ -137,25 +139,44 @@ public class TestDelegationToken {
         owner), new Text(renewer), null);
     return new Token<TestDelegationTokenIdentifier>(dtId, dtSecretManager);
   }
+  
+  private void shouldThrow(PrivilegedExceptionAction<Object> action,
+                           Class<? extends Throwable> except) {
+    try {
+      action.run();
+      Assert.fail("action did not throw " + except);
+    } catch (Throwable th) {
+      LOG.info("Caught an exception: " + StringUtils.stringifyException(th));
+      assertEquals("action threw wrong exception", except, th.getClass());
+    }
+  }
+
   @Test
   public void testDelegationTokenSecretManager() throws Exception {
-    TestDelegationTokenSecretManager dtSecretManager = 
+    final TestDelegationTokenSecretManager dtSecretManager = 
       new TestDelegationTokenSecretManager(24*60*60*1000,
           3*1000,1*1000,3600000);
     try {
       dtSecretManager.startThreads();
-      Token<TestDelegationTokenIdentifier> token = generateDelegationToken(
+      final Token<TestDelegationTokenIdentifier> token = 
+        generateDelegationToken(
           dtSecretManager, "SomeUser", "JobTracker");
       // Fake renewer should not be able to renew
-      Assert.assertFalse(dtSecretManager.renewToken(token, "FakeRenewer"));
-      Assert.assertTrue(dtSecretManager.renewToken(token, "JobTracker"));
+      shouldThrow(new PrivilegedExceptionAction<Object>() {
+        public Object run() throws Exception {
+          dtSecretManager.renewToken(token, "FakeRenewer");
+          return null;
+        }
+      }, AccessControlException.class);
+      long time = dtSecretManager.renewToken(token, "JobTracker");
+      assertTrue("renew time is in future", time > System.currentTimeMillis());
       TestDelegationTokenIdentifier identifier = 
         new TestDelegationTokenIdentifier();
       byte[] tokenId = token.getIdentifier();
       identifier.readFields(new DataInputStream(
           new ByteArrayInputStream(tokenId)));
       Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
-      Log.info("Sleep to expire the token");
+      LOG.info("Sleep to expire the token");
       Thread.sleep(2000);
       //Token should be expired
       try {
@@ -165,31 +186,49 @@ public class TestDelegationToken {
       } catch (InvalidToken e) {
         //Success
       }
-      Assert.assertTrue(dtSecretManager.renewToken(token, "JobTracker"));
-      Log.info("Sleep beyond the max lifetime");
+      dtSecretManager.renewToken(token, "JobTracker");
+      LOG.info("Sleep beyond the max lifetime");
       Thread.sleep(2000);
-      Assert.assertFalse(dtSecretManager.renewToken(token, "JobTracker"));
+      
+      shouldThrow(new PrivilegedExceptionAction<Object>() {
+        public Object run() throws Exception {
+          dtSecretManager.renewToken(token, "JobTracker");
+          return null;
+        }
+      }, InvalidToken.class);
     } finally {
       dtSecretManager.stopThreads();
     }
   }
+
   @Test 
   public void testCancelDelegationToken() throws Exception {
-    TestDelegationTokenSecretManager dtSecretManager = 
+    final TestDelegationTokenSecretManager dtSecretManager = 
       new TestDelegationTokenSecretManager(24*60*60*1000,
         10*1000,1*1000,3600000);
     try {
       dtSecretManager.startThreads();
-      Token<TestDelegationTokenIdentifier> token = generateDelegationToken(
-          dtSecretManager, "SomeUser", "JobTracker");
+      final Token<TestDelegationTokenIdentifier> token = 
+        generateDelegationToken(dtSecretManager, "SomeUser", "JobTracker");
       //Fake renewer should not be able to renew
-      Assert.assertFalse(dtSecretManager.cancelToken(token, "FakeCanceller"));
-      Assert.assertTrue(dtSecretManager.cancelToken(token, "JobTracker"));
-      Assert.assertFalse(dtSecretManager.renewToken(token, "JobTracker"));
+      shouldThrow(new PrivilegedExceptionAction<Object>() {
+        public Object run() throws Exception {
+          dtSecretManager.renewToken(token, "FakeCanceller");
+          return null;
+        }
+      }, AccessControlException.class);
+      dtSecretManager.cancelToken(token, "JobTracker");
+      shouldThrow(new PrivilegedExceptionAction<Object>() {
+        public Object run() throws Exception {
+          dtSecretManager.renewToken(token, "JobTracker");
+          return null;
+        }
+      }, InvalidToken.class);
     } finally {
       dtSecretManager.stopThreads();
     }
   }
+
   @Test
   public void testRollMasterKey() throws Exception {
     TestDelegationTokenSecretManager dtSecretManager = 
@@ -226,6 +265,7 @@ public class TestDelegationToken {
       dtSecretManager.stopThreads();
     }
   }
+
   @Test
   @SuppressWarnings("unchecked")
   public void testDelegationTokenSelector() throws Exception {