Return-Path: Delivered-To: apmail-hadoop-common-commits-archive@www.apache.org Received: (qmail 78818 invoked from network); 4 Mar 2011 03:48:34 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 4 Mar 2011 03:48:34 -0000 Received: (qmail 8527 invoked by uid 500); 4 Mar 2011 03:48:34 -0000 Delivered-To: apmail-hadoop-common-commits-archive@hadoop.apache.org Received: (qmail 8501 invoked by uid 500); 4 Mar 2011 03:48:34 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: common-dev@hadoop.apache.org Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 8494 invoked by uid 99); 4 Mar 2011 03:48:34 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 04 Mar 2011 03:48:34 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 04 Mar 2011 03:48:31 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 203D22388A2C; Fri, 4 Mar 2011 03:48:10 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1077168 - in /hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop: ipc/Client.java security/UserGroupInformation.java Date: Fri, 04 Mar 2011 03:48:10 -0000 To: common-commits@hadoop.apache.org From: omalley@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20110304034810.203D22388A2C@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: omalley Date: Fri Mar 4 03:48:09 2011 New Revision: 1077168 URL: http://svn.apache.org/viewvc?rev=1077168&view=rev Log: commit b5395c17f8ebb71a2b61510ed79839afbb4e133e Author: Devaraj Das Date: Sun Feb 14 23:34:54 2010 -0800 HADOOP:6559 from https://issues.apache.org/jira/secure/attachment/12435851/h-6559.6.bp20.patch +++ b/YAHOO-CHANGES.txt + HADOOP-6559. Makes the RPC client automatically re-login when the SASL connection + setup fails. This is applicable to only keytab based logins. (ddas) + Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/ipc/Client.java hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/UserGroupInformation.java Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/ipc/Client.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/ipc/Client.java?rev=1077168&r1=1077167&r2=1077168&view=diff ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/ipc/Client.java (original) +++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/ipc/Client.java Fri Mar 4 03:48:09 2011 @@ -343,6 +343,35 @@ public class Client { } } + private synchronized void setupSaslConnection(final InputStream in2, + final OutputStream out2) + throws javax.security.sasl.SaslException,IOException,InterruptedException { + try { + saslRpcClient = new SaslRpcClient(authMethod, token, + serverPrincipal); + saslRpcClient.saslConnect(in2, out2); + } catch (javax.security.sasl.SaslException je) { + if (authMethod == AuthMethod.KERBEROS && + UserGroupInformation.isLoginKeytabBased()) { + //try re-login + UserGroupInformation.getCurrentUser().reloginFromKeytab(); + //try setting up the connection again + try { + saslRpcClient = new SaslRpcClient(authMethod, token, + serverPrincipal); + saslRpcClient.saslConnect(in2, out2); + } catch (javax.security.sasl.SaslException jee) { + UserGroupInformation. + setLastUnsuccessfulAuthenticationAttemptTime + (System.currentTimeMillis()); + LOG.warn("Couldn't setup connection for " + + UserGroupInformation.getCurrentUser().getUserName() + + " to " + serverPrincipal + " even after relogin."); + throw jee; + } + } else throw je; + } + } /** Connect to the server and set up the I/O streams. It then sends * a header to the server and starts * the connection thread that waits for responses. @@ -389,10 +418,8 @@ public class Client { } ticket.doAs(new PrivilegedExceptionAction() { @Override - public Object run() throws IOException { - saslRpcClient = new SaslRpcClient(authMethod, token, - serverPrincipal); - saslRpcClient.saslConnect(in2, out2); + public Object run() throws IOException, InterruptedException { + setupSaslConnection(in2, out2); return null; } }); Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/UserGroupInformation.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/UserGroupInformation.java?rev=1077168&r1=1077167&r2=1077168&view=diff ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/UserGroupInformation.java (original) +++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/UserGroupInformation.java Fri Mar 4 03:48:09 2011 @@ -126,6 +126,10 @@ public class UserGroupInformation { private static boolean useKerberos; /** Server-side groups fetching service */ private static Groups groups; + /** The last authentication time */ + private static long lastUnsuccessfulAuthenticationAttemptTime; + + public 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 = @@ -198,6 +202,8 @@ public class UserGroupInformation { private final Subject subject; + private static LoginContext login; + private static final String OS_LOGIN_MODULE_NAME; private static final Class OS_PRINCIPAL_CLASS; private static final boolean windows = @@ -274,6 +280,7 @@ public class UserGroupInformation { static { USER_KERBEROS_OPTIONS.put("doNotPrompt", "true"); USER_KERBEROS_OPTIONS.put("useTicketCache", "true"); + USER_KERBEROS_OPTIONS.put("renewTGT", "true"); String ticketCache = System.getenv("KRB5CCNAME"); if (ticketCache != null) { USER_KERBEROS_OPTIONS.put("ticketCache", ticketCache); @@ -289,8 +296,6 @@ public class UserGroupInformation { KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true"); KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true"); KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true"); - KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", "true"); - KEYTAB_KERBEROS_OPTIONS.put("renewTGT", "true"); } private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN = new AppConfigurationEntry(Krb5LoginModule.class.getName(), @@ -351,7 +356,6 @@ public class UserGroupInformation { static UserGroupInformation getLoginUser() throws IOException { if (loginUser == null) { try { - LoginContext login; if (isSecurityEnabled()) { login = new LoginContext(HadoopConfiguration.USER_KERBEROS_CONFIG_NAME); } else { @@ -387,7 +391,7 @@ public class UserGroupInformation { keytabFile = path; keytabPrincipal = user; try { - LoginContext login = + login = new LoginContext(HadoopConfiguration.KEYTAB_KERBEROS_CONFIG_NAME); login.login(); loginUser = new UserGroupInformation(login.getSubject()); @@ -396,7 +400,57 @@ public class UserGroupInformation { path, le); } } + + /** + * Re-Login 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. This + * method assumes that {@link #loginUserFromKeytab(String, String)} 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 reloginFromKeytab() + throws IOException { + if (!isSecurityEnabled()) + return; + if (login == null || keytabFile == null) { + throw new IOException("loginUserFromKeyTab must be done first"); + } + if (System.currentTimeMillis() -lastUnsuccessfulAuthenticationAttemptTime < + 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; + } + 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); + login.login(); + } catch (LoginException le) { + throw new IOException("Login failure for " + keytabPrincipal + + " from keytab " + keytabFile, le); + } + } + public synchronized static void + setLastUnsuccessfulAuthenticationAttemptTime(long time) { + lastUnsuccessfulAuthenticationAttemptTime = time; + } + + public synchronized static boolean isLoginKeytabBased() { + return keytabFile != null; + } /** * 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.