HADOOP-6965. Introduces checks for whether the original tgt is valid in the reloginFromKeytab method. Contributed by Jitendra Pandey.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1002343 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Devaraj Das 2010-09-28 20:18:37 +00:00
parent 47d4ebebac
commit 8db4b543b9
2 changed files with 53 additions and 41 deletions

View File

@ -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(..).

View File

@ -498,6 +498,30 @@ static UserGroupInformation getLoginUser() throws IOException {
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 @@ private void spawnAutoRenewalThreadForUserCreds() {
!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 synchronized void reloginFromKeytab()
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);