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/07/29 19:29:52 UTC

svn commit: r980518 - in /hadoop/common/trunk: ./ src/java/org/apache/hadoop/ipc/ src/java/org/apache/hadoop/security/ src/test/core/org/apache/hadoop/security/

Author: ddas
Date: Thu Jul 29 17:29:51 2010
New Revision: 980518

URL: http://svn.apache.org/viewvc?rev=980518&view=rev
Log:
HADOOP-6656. Adds a thread in the UserGroupInformation to renew TGTs periodically. Contributed by Owen O'Malley and Devaraj Das.

Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Client.java
    hadoop/common/trunk/src/java/org/apache/hadoop/security/SecurityUtil.java
    hadoop/common/trunk/src/java/org/apache/hadoop/security/User.java
    hadoop/common/trunk/src/java/org/apache/hadoop/security/UserGroupInformation.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/security/TestSecurityUtil.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/security/TestUserGroupInformation.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=980518&r1=980517&r2=980518&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Thu Jul 29 17:29:51 2010
@@ -94,6 +94,9 @@ Trunk (unreleased changes)
     HADOOP-6475. Adding some javadoc to Server.RpcMetrics, UGI. 
     (Jitendra Pandey and borya via jghoman)
 
+    HADOOP-6656. Adds a thread in the UserGroupInformation to renew TGTs 
+    periodically. (Owen O'Malley and ddas via ddas)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Client.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Client.java?rev=980518&r1=980517&r2=980518&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Client.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Client.java Thu Jul 29 17:29:51 2010
@@ -381,18 +381,22 @@ public class Client {
         return saslRpcClient.saslConnect(in2, out2);
       } catch (javax.security.sasl.SaslException je) {
         UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
-        UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+        UserGroupInformation currentUser = 
+          UserGroupInformation.getCurrentUser();
         UserGroupInformation realUser = currentUser.getRealUser();
         if (authMethod == AuthMethod.KERBEROS && 
           //try setting up the connection again
-          UserGroupInformation.isLoginKeytabBased() &&
           // relogin only in case it is the login user (e.g. JT)
           // or superuser (like oozie).
           ((currentUser != null && currentUser.equals(loginUser)) ||
            (realUser != null && realUser.equals(loginUser)))) {
           try {
             //try re-login
-            loginUser.reloginFromKeytab();
+            if (UserGroupInformation.isLoginKeytabBased()) {
+              loginUser.reloginFromKeytab();
+            } else {
+              loginUser.reloginFromTicketCache();
+            }
             disposeSasl();
             saslRpcClient = new SaslRpcClient(authMethod, token,
                 serverPrincipal);

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/security/SecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/SecurityUtil.java?rev=980518&r1=980517&r2=980518&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/SecurityUtil.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/SecurityUtil.java Thu Jul 29 17:29:51 2010
@@ -25,6 +25,7 @@ import java.security.AccessController;
 import java.util.Set;
 
 import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
 import javax.security.auth.kerberos.KerberosTicket;
 
 import org.apache.commons.logging.Log;
@@ -62,21 +63,36 @@ public class SecurityUtil {
     Set<KerberosTicket> tickets = current
         .getPrivateCredentials(KerberosTicket.class);
     for (KerberosTicket t : tickets) {
-      if (isOriginalTGT(t.getServer().getName()))
+      if (isOriginalTGT(t))
         return t;
     }
     throw new IOException("Failed to find TGT from current Subject");
   }
   
-  // Original TGT must be of form "krbtgt/FOO@FOO". Verify this
-  protected static boolean isOriginalTGT(String name) {
-    if(name == null) return false;
-    
-    String [] components = name.split("[/@]");
-
-    return components.length == 3 &&
-           "krbtgt".equals(components[0]) &&
-           components[1].equals(components[2]);
+  /**
+   * TGS must have the server principal of the form "krbtgt/FOO@FOO".
+   * @param principal
+   * @return true or false
+   */
+  static boolean 
+  isTGSPrincipal(KerberosPrincipal principal) {
+    if (principal == null)
+      return false;
+    if (principal.getName().equals("krbtgt/" + principal.getRealm() + 
+        "@" + principal.getRealm())) {
+      return true;
+    }
+    return false;
+  }
+  
+  /**
+   * Check whether the server principal is the TGS's principal
+   * @param ticket the original TGT (the ticket that is obtained when a 
+   * kinit is done)
+   * @return true or false
+   */
+  protected static boolean isOriginalTGT(KerberosTicket ticket) {
+    return isTGSPrincipal(ticket.getServer());
   }
 
   /**

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/security/User.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/User.java?rev=980518&r1=980517&r2=980518&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/User.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/User.java Thu Jul 29 17:29:51 2010
@@ -35,8 +35,9 @@ import org.apache.hadoop.security.UserGr
 class User implements Principal {
   private final String fullName;
   private final String shortName;
-  private AuthenticationMethod authMethod = null;
-  private LoginContext login = null;
+  private volatile AuthenticationMethod authMethod = null;
+  private volatile LoginContext login = null;
+  private volatile long lastLogin = 0;
 
   public User(String name) {
     this(name, null, null);
@@ -114,4 +115,20 @@ class User implements Principal {
   public void setLogin(LoginContext login) {
     this.login = login;
   }
+  
+  /**
+   * Set the last login time.
+   * @param time the number of milliseconds since the beginning of time
+   */
+  public void setLastLogin(long time) {
+    lastLogin = time;
+  }
+  
+  /**
+   * Get the time of the last login.
+   * @return the number of milliseconds since the beginning of time.
+   */
+  public long getLastLogin() {
+    return lastLogin;
+  }
 }

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=980518&r1=980517&r2=980518&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 Thu Jul 29 17:29:51 2010
@@ -38,7 +38,9 @@ import java.util.Set;
 
 import javax.security.auth.Subject;
 import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.kerberos.KerberosKey;
 import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.kerberos.KerberosTicket;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.LoginContext;
 import javax.security.auth.login.LoginException;
@@ -53,6 +55,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.Shell;
 
 import com.sun.security.auth.NTUserPrincipal;
 import com.sun.security.auth.UnixPrincipal;
@@ -68,6 +71,10 @@ import com.sun.security.auth.module.Krb5
 @InterfaceStability.Evolving
 public class UserGroupInformation {
   private static final Log LOG =  LogFactory.getLog(UserGroupInformation.class);
+  /**
+   * Percentage of the ticket window to use before we renew ticket.
+   */
+  private static final float TICKET_RENEW_WINDOW = 0.80f;
   
   /**
    * A login module that looks at the Kerberos, Unix, or Windows principal and
@@ -140,15 +147,13 @@ public class UserGroupInformation {
   private static Configuration conf;
 
   
-  public static final long MIN_TIME_BEFORE_RELOGIN = 10 * 60 * 1000L;
+  /** Leave 10 minutes between relogin attempts. */
+  private static final long MIN_TIME_BEFORE_RELOGIN = 10 * 60 * 1000L;
   
   /**Environment variable pointing to the token cache file*/
   public static final String HADOOP_TOKEN_FILE_LOCATION = 
     "HADOOP_TOKEN_FILE_LOCATION";
   
-  /** The last relogin attempt */
-  private long lastReloginTime = 0;
-
   /** 
    * A method to initialize the fields that depend on a configuration.
    * Must be called before useKerberos or groups is used.
@@ -224,6 +229,9 @@ public class UserGroupInformation {
   private static String keytabFile = null;
 
   private final Subject subject;
+  // All non-static fields must be read-only caches that come from the subject.
+  private final User user;
+  private final boolean isKeytab;
   
   private static final String OS_LOGIN_MODULE_NAME;
   private static final Class<? extends Principal> OS_PRINCIPAL_CLASS;
@@ -349,16 +357,11 @@ public class UserGroupInformation {
   }
   
   private LoginContext getLogin() {
-    for (User p: subject.getPrincipals(User.class)) {
-      return p.getLogin();
-    }
-    return null;
+    return user.getLogin();
   }
   
   private void setLogin(LoginContext login) {
-    for (User p: subject.getPrincipals(User.class)) {
-      p.setLogin(login);
-    }
+    user.setLogin(login);
   }
 
   /**
@@ -368,6 +371,8 @@ public class UserGroupInformation {
    */
   UserGroupInformation(Subject subject) {
     this.subject = subject;
+    this.user = subject.getPrincipals(User.class).iterator().next();
+    this.isKeytab = !subject.getPrivateCredentials(KerberosKey.class).isEmpty();
   }
 
   /**
@@ -391,7 +396,6 @@ public class UserGroupInformation {
     if (loginUser == null) {
       try {
         Subject subject = new Subject();
-        loginUser = new UserGroupInformation(subject);
         LoginContext login;
         if (isSecurityEnabled()) {
           login = new LoginContext(HadoopConfiguration.USER_KERBEROS_CONFIG_NAME,
@@ -401,7 +405,11 @@ public class UserGroupInformation {
               subject);
         }
         login.login();
+        loginUser = new UserGroupInformation(subject);
         loginUser.setLogin(login);
+        loginUser.setAuthenticationMethod(isSecurityEnabled() ?
+                                          AuthenticationMethod.KERBEROS :
+                                          AuthenticationMethod.SIMPLE);
         loginUser = new UserGroupInformation(login.getSubject());
         String fileLocation = System.getenv(HADOOP_TOKEN_FILE_LOCATION);
         if (fileLocation != null && isSecurityEnabled()) {
@@ -413,6 +421,7 @@ public class UserGroupInformation {
             loginUser.addToken(token);
           }
         }
+        loginUser.spawnAutoRenewalThreadForUserCreds();
       } catch (LoginException le) {
         throw new IOException("failure to login", le);
       }
@@ -421,6 +430,90 @@ public class UserGroupInformation {
   }
 
   /**
+   * Is this user logged in from a keytab file?
+   * @return true if the credentials are from a keytab file.
+   */
+  public boolean isFromKeytab() {
+    return isKeytab;
+  }
+
+  /**Spawn a thread to do periodic renewals of kerberos credentials*/
+  private void spawnAutoRenewalThreadForUserCreds() {
+    if (isSecurityEnabled()) {
+      //spawn thread only if we have kerb credentials
+      if (user.getAuthenticationMethod() == AuthenticationMethod.KERBEROS &&
+          !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");
+            KerberosTicket tgt = getTGT();
+            if (tgt == null) {
+              return;
+            }
+            long nextRefresh = getRefreshTime(tgt);
+            while (true) {
+              try {
+                long now = System.currentTimeMillis();
+                LOG.debug("Current time is " + now);
+                LOG.debug("Next refresh is " + nextRefresh);
+                if (now < nextRefresh) {
+                  Thread.sleep(nextRefresh - now);
+                }
+                Shell.execCommand(cmd, "-R");
+                LOG.debug("renewed ticket");
+                reloginFromTicketCache();
+                tgt = getTGT();
+                if (tgt == null) {
+                  LOG.warn("No TGT after renewal. Aborting renew thread for " +
+                           getUserName());
+                  return;
+                }
+                nextRefresh = Math.max(getRefreshTime(tgt),
+                                       now + MIN_TIME_BEFORE_RELOGIN);
+              } catch (InterruptedException ie) {
+                LOG.warn("Terminating renewal thread");
+                return;
+              } catch (IOException ie) {
+                LOG.warn("Exception encountered while running the" +
+                    " renewal command. Aborting renew thread. " + ie);
+                return;
+              }
+            }
+          }
+        });
+        t.setDaemon(true);
+        t.setName("TGT Renewer for " + getUserName());
+        t.start();
+      }
+    }
+  }
+  /**
    * Log a user in from a keytab file. Loads a user identity from a keytab
    * file and login them in. They become the currently logged-in user.
    * @param user the principal name to load from the keytab
@@ -444,6 +537,7 @@ public class UserGroupInformation {
       login.login();
       loginUser = new UserGroupInformation(subject);
       loginUser.setLogin(login);
+      loginUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
     } catch (LoginException le) {
       throw new IOException("Login failure for " + user + " from keytab " + 
                             path, le);
@@ -463,21 +557,20 @@ public class UserGroupInformation {
    */
   public synchronized void reloginFromKeytab()
   throws IOException {
-    if (!isSecurityEnabled())
+    if (!isSecurityEnabled() ||
+         user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS ||
+         !isKeytab)
       return;
     LoginContext login = getLogin();
     if (login == null || keytabFile == null) {
       throw new IOException("loginUserFromKeyTab must be done first");
     }
     long now = System.currentTimeMillis();
-    if (now - lastReloginTime < MIN_TIME_BEFORE_RELOGIN ) {
-      LOG.warn("Not attempting to re-login since the last re-login was " +
-          "attempted less than " + (MIN_TIME_BEFORE_RELOGIN/1000) + " seconds"+
-          " before.");
+    if (!hasSufficientTimeElapsed(now)) {
       return;
     }
-    // register most recent relogin
-    lastReloginTime = System.currentTimeMillis();
+    // 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 
@@ -499,6 +592,49 @@ public class UserGroupInformation {
   }
 
   /**
+   * Re-Login a user in from the ticket cache.  This
+   * method assumes that login had happened already.
+   * The Subject field of this UserGroupInformation object is updated to have
+   * the new credentials.
+   * @throws IOException on a failure
+   */
+  public synchronized void reloginFromTicketCache()
+  throws IOException {
+    if (!isSecurityEnabled() || 
+        user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS ||
+        isKeytab)
+      return;
+    LoginContext login = getLogin();
+    if (login == null) {
+      throw new IOException("login must be done first");
+    }
+    long now = System.currentTimeMillis();
+    if (!hasSufficientTimeElapsed(now)) {
+      return;
+    }
+    // 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.USER_KERBEROS_CONFIG_NAME, 
+            getSubject());
+      LOG.info("Initiating re-login for " + getUserName());
+      login.login();
+      setLogin(login);
+    } catch (LoginException le) {
+      throw new IOException("Login failure for " + getUserName(), le);
+    } 
+  }
+
+
+  /**
    * Log a user in from a keytab file. Loads a user identity from a keytab
    * file and login them in. This new user does not affect the currently
    * logged-in user.
@@ -527,6 +663,7 @@ public class UserGroupInformation {
       login.login();
       UserGroupInformation newLoginUser = new UserGroupInformation(subject);
       newLoginUser.setLogin(login);
+      newLoginUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
       
       return newLoginUser;
     } catch (LoginException le) {
@@ -538,9 +675,24 @@ public class UserGroupInformation {
     }
   }
 
+  private boolean hasSufficientTimeElapsed(long now) {
+    if (now - user.getLastLogin() < MIN_TIME_BEFORE_RELOGIN ) {
+      LOG.warn("Not attempting to re-login since the last re-login was " +
+          "attempted less than " + (MIN_TIME_BEFORE_RELOGIN/1000) + " seconds"+
+          " before.");
+      return false;
+    }
+    return true;
+  }
+  
+  /**
+   * Did the login happen via keytab
+   * @return true or false
+   */
   public synchronized static boolean isLoginKeytabBased() {
-    return keytabFile != null;
+    return loginUser.isKeytab;
   }
+
   /**
    * Create a user from a login name. It is intended to be used for remote
    * users in RPC, since it won't have any credentials.
@@ -553,7 +705,9 @@ public class UserGroupInformation {
     }
     Subject subject = new Subject();
     subject.getPrincipals().add(new User(user));
-    return new UserGroupInformation(subject);
+    UserGroupInformation result = new UserGroupInformation(subject);
+    result.setAuthenticationMethod(AuthenticationMethod.SIMPLE);
+    return result;
   }
 
   /**
@@ -586,9 +740,12 @@ public class UserGroupInformation {
       throw new IllegalArgumentException("Null real user");
     }
     Subject subject = new Subject();
-    subject.getPrincipals().add(new User(user));
-    subject.getPrincipals().add(new RealUser(realUser));
-    return new UserGroupInformation(subject);
+    Set<Principal> principals = subject.getPrincipals();
+    principals.add(new User(user));
+    principals.add(new RealUser(realUser));
+    UserGroupInformation result =new UserGroupInformation(subject);
+    result.setAuthenticationMethod(AuthenticationMethod.PROXY);
+    return result;
   }
 
   /**
@@ -692,10 +849,7 @@ public class UserGroupInformation {
    * @return the user's full principal name.
    */
   public String getUserName() {
-    for (User p: subject.getPrincipals(User.class)) {
-      return p.getName();
-    }
-    return null;
+    return user.getName();
   }
 
   /**
@@ -782,9 +936,7 @@ public class UserGroupInformation {
    */
   public synchronized 
   void setAuthenticationMethod(AuthenticationMethod authMethod) {
-    for (User p : subject.getPrincipals(User.class)) {
-      p.setAuthenticationMethod(authMethod);
-    }
+    user.setAuthenticationMethod(authMethod);
   }
 
   /**
@@ -793,10 +945,7 @@ public class UserGroupInformation {
    * @return AuthenticationMethod in the subject, null if not present.
    */
   public synchronized AuthenticationMethod getAuthenticationMethod() {
-    for (User p: subject.getPrincipals(User.class)) {
-      return p.getAuthenticationMethod();
-    }
-    return null;
+    return user.getAuthenticationMethod();
   }
   
   /**
@@ -909,6 +1058,8 @@ public class UserGroupInformation {
     UserGroupInformation ugi = getCurrentUser();
     ugi.print();
     System.out.println("UGI: " + ugi);
+    System.out.println("Auth method " + ugi.user.getAuthenticationMethod());
+    System.out.println("Keytab " + ugi.isKeytab);
     System.out.println("============================================================");
     
     if (args.length == 2) {
@@ -916,6 +1067,8 @@ public class UserGroupInformation {
       loginUserFromKeytab(args[0], args[1]);
       getCurrentUser().print();
       System.out.println("Keytab: " + ugi);
+      System.out.println("Auth method " + loginUser.user.getAuthenticationMethod());
+      System.out.println("Keytab " + loginUser.isKeytab);
     }
   }
 

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/security/TestSecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/security/TestSecurityUtil.java?rev=980518&r1=980517&r2=980518&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/security/TestSecurityUtil.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/security/TestSecurityUtil.java Thu Jul 29 17:29:51 2010
@@ -20,20 +20,29 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 
+import javax.security.auth.kerberos.KerberosPrincipal;
+
 import org.junit.Test;
 
 public class TestSecurityUtil {
   @Test
   public void isOriginalTGTReturnsCorrectValues() {
-    assertTrue(SecurityUtil.isOriginalTGT("krbtgt/foo@foo"));
-    assertTrue(SecurityUtil.isOriginalTGT("krbtgt/foo.bar.bat@foo.bar.bat"));
-    assertFalse(SecurityUtil.isOriginalTGT(null));
-    assertFalse(SecurityUtil.isOriginalTGT("blah"));
-    assertFalse(SecurityUtil.isOriginalTGT(""));
-    assertFalse(SecurityUtil.isOriginalTGT("krbtgt/hello"));
-    assertFalse(SecurityUtil.isOriginalTGT("/@"));
-    assertFalse(SecurityUtil.isOriginalTGT("this@is/notright"));
-    assertFalse(SecurityUtil.isOriginalTGT("krbtgt/foo@FOO"));
+    assertTrue(SecurityUtil.isTGSPrincipal
+        (new KerberosPrincipal("krbtgt/foo@foo")));
+    assertTrue(SecurityUtil.isTGSPrincipal
+        (new KerberosPrincipal("krbtgt/foo.bar.bat@foo.bar.bat")));
+    assertFalse(SecurityUtil.isTGSPrincipal
+        (null));
+    assertFalse(SecurityUtil.isTGSPrincipal
+        (new KerberosPrincipal("blah")));
+    assertFalse(SecurityUtil.isTGSPrincipal
+        (new KerberosPrincipal("")));
+    assertFalse(SecurityUtil.isTGSPrincipal
+        (new KerberosPrincipal("krbtgt/hello")));
+    assertFalse(SecurityUtil.isTGSPrincipal
+        (new KerberosPrincipal("/@")));
+    assertFalse(SecurityUtil.isTGSPrincipal
+        (new KerberosPrincipal("krbtgt/foo@FOO")));
   }
   
   private void verify(String original, String hostname, String expected)

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/security/TestUserGroupInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/security/TestUserGroupInformation.java?rev=980518&r1=980517&r2=980518&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/security/TestUserGroupInformation.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/security/TestUserGroupInformation.java Thu Jul 29 17:29:51 2010
@@ -279,8 +279,8 @@ public class TestUserGroupInformation {
     final AuthenticationMethod am = AuthenticationMethod.KERBEROS;
     ugi.setAuthenticationMethod(am);
     Assert.assertEquals(am, ugi.getAuthenticationMethod());
-    Assert.assertEquals(null, proxyUgi.getAuthenticationMethod());
-    proxyUgi.setAuthenticationMethod(AuthenticationMethod.PROXY);
+    Assert.assertEquals(AuthenticationMethod.PROXY,
+                        proxyUgi.getAuthenticationMethod());
     Assert.assertEquals(am, UserGroupInformation
         .getRealAuthenticationMethod(proxyUgi));
     proxyUgi.doAs(new PrivilegedExceptionAction<Object>() {