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 dd...@apache.org on 2010/09/28 22:18:38 UTC

svn commit: r1002343 - in /hadoop/common/trunk: CHANGES.txt src/java/org/apache/hadoop/security/UserGroupInformation.java

Author: ddas
Date: Tue Sep 28 20:18:37 2010
New Revision: 1002343

URL: http://svn.apache.org/viewvc?rev=1002343&view=rev
Log:
HADOOP-6965. Introduces checks for whether the original tgt is valid in the reloginFromKeytab method. Contributed by Jitendra Pandey.

Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/org/apache/hadoop/security/UserGroupInformation.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=1002343&r1=1002342&r2=1002343&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Tue Sep 28 20:18:37 2010
@@ -134,6 +134,9 @@ Trunk (unreleased changes)
     HADOOP-6922. Make AccessControlList a writable and update documentation
     for Job ACLs.  (Ravi Gummadi via vinodkv)
 
+    HADOOP-6965. Introduces checks for whether the original tgt is valid 
+    in the reloginFromKeytab method.
+
   OPTIMIZATIONS
 
     HADOOP-6884. Add LOG.isDebugEnabled() guard for each LOG.debug(..).

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/security/UserGroupInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/UserGroupInformation.java?rev=1002343&r1=1002342&r2=1002343&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/UserGroupInformation.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/UserGroupInformation.java Tue Sep 28 20:18:37 2010
@@ -498,6 +498,30 @@ public class UserGroupInformation {
   public boolean isFromKeytab() {
     return isKeytab;
   }
+  
+  /**
+   * Get the Kerberos TGT
+   * @return the user's TGT or null if none was found
+   */
+  private KerberosTicket getTGT() {
+    Set<KerberosTicket> tickets = subject
+        .getPrivateCredentials(KerberosTicket.class);
+    for (KerberosTicket ticket : tickets) {
+      if (SecurityUtil.isOriginalTGT(ticket)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Found tgt " + ticket);
+        }
+        return ticket;
+      }
+    }
+    return null;
+  }
+  
+  private long getRefreshTime(KerberosTicket tgt) {
+    long start = tgt.getStartTime().getTime();
+    long end = tgt.getEndTime().getTime();
+    return start + (long) ((end - start) * TICKET_RENEW_WINDOW);
+  }
 
   /**Spawn a thread to do periodic renewals of kerberos credentials*/
   private void spawnAutoRenewalThreadForUserCreds() {
@@ -507,30 +531,6 @@ public class UserGroupInformation {
           !isKeytab) {
         Thread t = new Thread(new Runnable() {
           
-          /**
-           * Get the Kerberos TGT
-           * @return the user's TGT or null if none was found
-           */
-          private KerberosTicket getTGT() {
-            Set<KerberosTicket> tickets = 
-              subject.getPrivateCredentials(KerberosTicket.class);
-            for(KerberosTicket ticket: tickets) {
-              if (SecurityUtil.isOriginalTGT(ticket)) {
-                if (LOG.isDebugEnabled()) {
-                  LOG.debug("Found tgt " + ticket);
-                }
-                return ticket;
-              }
-            }
-            return null;
-          }
-
-          private long getRefreshTime(KerberosTicket tgt) {
-            long start = tgt.getStartTime().getTime();
-            long end = tgt.getEndTime().getTime();
-            return start + (long) ((end - start) * TICKET_RENEW_WINDOW);
-          }
-
           public void run() {
             String cmd = conf.get("hadoop.kerberos.kinit.command",
                                   "/usr/kerberos/bin/kinit");
@@ -633,33 +633,42 @@ public class UserGroupInformation {
          user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS ||
          !isKeytab)
       return;
+    
+    KerberosTicket tgt = getTGT();
+    if (tgt == null) {
+      return;
+    }
+    //Return if TGT is valid and is not going to expire soon.
+    if (System.currentTimeMillis() < getRefreshTime(tgt)) {
+      return;
+    }
+    
     LoginContext login = getLogin();
     if (login == null || keytabFile == null) {
       throw new IOException("loginUserFromKeyTab must be done first");
     }
     long now = System.currentTimeMillis();
-    if (!hasSufficientTimeElapsed(now)) {
-      return;
-    }
+    
     long start = 0;
     // register most recent relogin attempt
     user.setLastLogin(now);
     try {
       LOG.info("Initiating logout for " + getUserName());
-      //clear up the kerberos state. But the tokens are not cleared! As per 
-      //the Java kerberos login module code, only the kerberos credentials
-      //are cleared
-      login.logout();
-      //login and also update the subject field of this instance to 
-      //have the new credentials (pass it to the LoginContext constructor)
-      login = 
-        new LoginContext(HadoopConfiguration.KEYTAB_KERBEROS_CONFIG_NAME, 
-            getSubject());
-      LOG.info("Initiating re-login for " + keytabPrincipal);
-      start = System.currentTimeMillis();
-      login.login();
-      metrics.loginSuccess.inc(System.currentTimeMillis() - start);
-      setLogin(login);
+      synchronized (UserGroupInformation.class) {
+        // clear up the kerberos state. But the tokens are not cleared! As per
+        // the Java kerberos login module code, only the kerberos credentials
+        // are cleared
+        login.logout();
+        // login and also update the subject field of this instance to
+        // have the new credentials (pass it to the LoginContext constructor)
+        login = new LoginContext(
+            HadoopConfiguration.KEYTAB_KERBEROS_CONFIG_NAME, getSubject());
+        LOG.info("Initiating re-login for " + keytabPrincipal);
+        start = System.currentTimeMillis();
+        login.login();
+        metrics.loginSuccess.inc(System.currentTimeMillis() - start);
+        setLogin(login);
+      }
     } catch (LoginException le) {
       if (start > 0) {
         metrics.loginFailure.inc(System.currentTimeMillis() - start);