[ 
https://issues.apache.org/jira/browse/HADOOP-16122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16772596#comment-16772596
 ] 

chendihao edited comment on HADOOP-16122 at 2/20/19 4:03 AM:
-------------------------------------------------------------

Here is the code which leads to this problem.

In `UserGroupInformation.java`, The `keytabPrincipal` and `keytabFile` are 
static properties. So all the UGI objects in the same process use these same 
properties.
{code:java}
public class UserGroupInformation {
    private static UserGroupInformation loginUser = null;
    private static String keytabPrincipal = null;
    private static String keytabFile = null;
}{code}
When we invoke `reloginFromKeytab()`, we use "hadoop-keytab-kerberos" to create 
the `LoginContext`.
{code:java}
public synchronized void reloginFromKeytab() throws IOException {
    if(isSecurityEnabled() && this.user.getAuthenticationMethod() == 
UserGroupInformation.AuthenticationMethod.KERBEROS && this.isKeytab) {
        long now = Time.now();
        if(shouldRenewImmediatelyForTests || 
this.hasSufficientTimeElapsed(now)) {
            KerberosTicket tgt = this.getTGT();
            if(tgt == null || shouldRenewImmediatelyForTests || now >= 
this.getRefreshTime(tgt)) {
                LoginContext login = this.getLogin();
                if(login != null && keytabFile != null) {
                    long start = 0L;
                    this.user.setLastLogin(now);

                    try {
                        if(LOG.isDebugEnabled()) {
                            LOG.debug("Initiating logout for " + 
this.getUserName());
                        }

                        Class var7 = UserGroupInformation.class;
                        synchronized(UserGroupInformation.class) {
                            login.logout();
                            login = newLoginContext("hadoop-keytab-kerberos", 
this.getSubject(), new UserGroupInformation.HadoopConfiguration(null));
                            if(LOG.isDebugEnabled()) {
                                LOG.debug("Initiating re-login for " + 
keytabPrincipal);
                            }

                            start = Time.now();
                            login.login();
                            metrics.loginSuccess.add(Time.now() - start);
                            this.setLogin(login);
                        }
                    } catch (LoginException var10) {
                        if(start > 0L) {
                            metrics.loginFailure.add(Time.now() - start);
                        }

                        throw new IOException("Login failure for " + 
keytabPrincipal + " from keytab " + keytabFile, var10);
                    }
                } else {
                    throw new IOException("loginUserFromKeyTab must be done 
first");
                }
            }
        }
    }
} {code}
In the implementation of `HadoopConfiguration.getAppConfigurationEntry()`, it 
uses the static `keytabFile` and `keytabPrincipal` for "hadoop-keytab-kerberos".
{code:java}
public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
    if("hadoop-simple".equals(appName)) {
        return SIMPLE_CONF;
    } else if("hadoop-user-kerberos".equals(appName)) {
        return USER_KERBEROS_CONF;
    } else if("hadoop-keytab-kerberos".equals(appName)) {
        if(PlatformName.IBM_JAVA) {
            KEYTAB_KERBEROS_OPTIONS.put("useKeytab", 
UserGroupInformation.prependFileAuthority(UserGroupInformation.keytabFile));
        } else {
            KEYTAB_KERBEROS_OPTIONS.put("keyTab", 
UserGroupInformation.keytabFile);
        }

        KEYTAB_KERBEROS_OPTIONS.put("principal", 
UserGroupInformation.keytabPrincipal);
        return KEYTAB_KERBEROS_CONF;
    } else {
        return null;
    }
}
{code}
And the static `keytabFile` and `keytabPrincipal` are alway from the first 
login UGI. Here is the code of `loginUserFromKeytabAndReturnUGI()` and finally 
it update the static keytabFile and keytabFilePrincipal with the last one if it 
exists.
{code:java}
public static synchronized UserGroupInformation 
loginUserFromKeytabAndReturnUGI(String user, String path) throws IOException {
    if(!isSecurityEnabled()) {
        return getCurrentUser();
    } else {
        String oldKeytabFile = null;
        String oldKeytabPrincipal = null;
        long start = 0L;

        UserGroupInformation var9;
        try {
            oldKeytabFile = keytabFile;
            oldKeytabPrincipal = keytabPrincipal;
            keytabFile = path;
            keytabPrincipal = user;
            Subject subject = new Subject();
            LoginContext login = newLoginContext("hadoop-keytab-kerberos", 
subject, new UserGroupInformation.HadoopConfiguration(null));
            start = Time.now();
            login.login();
            metrics.loginSuccess.add(Time.now() - start);
            UserGroupInformation newLoginUser = new 
UserGroupInformation(subject);
            newLoginUser.setLogin(login);
            
newLoginUser.setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.KERBEROS);
            var9 = newLoginUser;
        } catch (LoginException var13) {
            if(start > 0L) {
                metrics.loginFailure.add(Time.now() - start);
            }

            throw new IOException("Login failure for " + user + " from keytab " 
+ path, var13);
        } finally {
            if(oldKeytabFile != null) {
                keytabFile = oldKeytabFile;
            }

            if(oldKeytabPrincipal != null) {
                keytabPrincipal = oldKeytabPrincipal;
            }

        }

        return var9;
    }
}
{code}
It means that different UGI objects can call non-static `reloginFromKeytab()` 
but always use the first login UGI's keytab and keytabPrincipal to pass to 
`Krb5LoginModule`.

For the first time, `loginUserFromKeytabAndReturnUGI` works because it updates 
the global static keytab and keytabPrincipal before creating `LoginContext` 
with "hadoop-keytab-kerberos". But it throws these properties for all UGIs(only 
keep the first one in static) and the logic of login can not do that correctly.


was (Author: tobe):
Here is the code which leads to this problem.

In `UserGroupInformation.java`, The `keytabPrincipal` and `keytabFile` are 
static properties. So all the UGI objects in the same process use these same 
properties.
{code:java}
public class UserGroupInformation {
    private static UserGroupInformation loginUser = null;
    private static String keytabPrincipal = null;
    private static String keytabFile = null;
}{code}
When we invoke `reloginFromKeytab()`, we use "hadoop-keytab-kerberos" to create 
the `LoginContext`.
{code:java}
public synchronized void reloginFromKeytab() throws IOException {
    if(isSecurityEnabled() && this.user.getAuthenticationMethod() == 
UserGroupInformation.AuthenticationMethod.KERBEROS && this.isKeytab) {
        long now = Time.now();
        if(shouldRenewImmediatelyForTests || 
this.hasSufficientTimeElapsed(now)) {
            KerberosTicket tgt = this.getTGT();
            if(tgt == null || shouldRenewImmediatelyForTests || now >= 
this.getRefreshTime(tgt)) {
                LoginContext login = this.getLogin();
                if(login != null && keytabFile != null) {
                    long start = 0L;
                    this.user.setLastLogin(now);

                    try {
                        if(LOG.isDebugEnabled()) {
                            LOG.debug("Initiating logout for " + 
this.getUserName());
                        }

                        Class var7 = UserGroupInformation.class;
                        synchronized(UserGroupInformation.class) {
                            login.logout();
                            login = newLoginContext("hadoop-keytab-kerberos", 
this.getSubject(), new UserGroupInformation.HadoopConfiguration(null));
                            if(LOG.isDebugEnabled()) {
                                LOG.debug("Initiating re-login for " + 
keytabPrincipal);
                            }

                            start = Time.now();
                            login.login();
                            metrics.loginSuccess.add(Time.now() - start);
                            this.setLogin(login);
                        }
                    } catch (LoginException var10) {
                        if(start > 0L) {
                            metrics.loginFailure.add(Time.now() - start);
                        }

                        throw new IOException("Login failure for " + 
keytabPrincipal + " from keytab " + keytabFile, var10);
                    }
                } else {
                    throw new IOException("loginUserFromKeyTab must be done 
first");
                }
            }
        }
    }
} {code}
In the implementation of `HadoopConfiguration.getAppConfigurationEntry()`, it 
uses the static `keytabFile` and `keytabPrincipal` for "hadoop-keytab-kerberos".
{code:java}
public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
    if("hadoop-simple".equals(appName)) {
        return SIMPLE_CONF;
    } else if("hadoop-user-kerberos".equals(appName)) {
        return USER_KERBEROS_CONF;
    } else if("hadoop-keytab-kerberos".equals(appName)) {
        if(PlatformName.IBM_JAVA) {
            KEYTAB_KERBEROS_OPTIONS.put("useKeytab", 
UserGroupInformation.prependFileAuthority(UserGroupInformation.keytabFile));
        } else {
            KEYTAB_KERBEROS_OPTIONS.put("keyTab", 
UserGroupInformation.keytabFile);
        }

        KEYTAB_KERBEROS_OPTIONS.put("principal", 
UserGroupInformation.keytabPrincipal);
        return KEYTAB_KERBEROS_CONF;
    } else {
        return null;
    }
}
{code}
And the static `keytabFile` and `keytabPrincipal` are alway from the first 
login UGI. Here is the code of `loginUserFromKeytabAndReturnUGI()` and finally 
it update the static keytabFile and keytabFilePrincipal with the last one if it 
exists.
{code:java}
public static synchronized UserGroupInformation 
loginUserFromKeytabAndReturnUGI(String user, String path) throws IOException {
    if(!isSecurityEnabled()) {
        return getCurrentUser();
    } else {
        String oldKeytabFile = null;
        String oldKeytabPrincipal = null;
        long start = 0L;

        UserGroupInformation var9;
        try {
            oldKeytabFile = keytabFile;
            oldKeytabPrincipal = keytabPrincipal;
            keytabFile = path;
            keytabPrincipal = user;
            Subject subject = new Subject();
            LoginContext login = newLoginContext("hadoop-keytab-kerberos", 
subject, new UserGroupInformation.HadoopConfiguration(null));
            start = Time.now();
            login.login();
            metrics.loginSuccess.add(Time.now() - start);
            UserGroupInformation newLoginUser = new 
UserGroupInformation(subject);
            newLoginUser.setLogin(login);
            
newLoginUser.setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.KERBEROS);
            var9 = newLoginUser;
        } catch (LoginException var13) {
            if(start > 0L) {
                metrics.loginFailure.add(Time.now() - start);
            }

            throw new IOException("Login failure for " + user + " from keytab " 
+ path, var13);
        } finally {
            if(oldKeytabFile != null) {
                keytabFile = oldKeytabFile;
            }

            if(oldKeytabPrincipal != null) {
                keytabPrincipal = oldKeytabPrincipal;
            }

        }

        return var9;
    }
}
{code}
It means that different UGI objects can call non-static `reloginFromKeytab()` 
but always use the first login UGI's keytab and keytabPrincipal to pass to 
`Krb5LoginModule`.

> Re-login from keytab for multiple Hadoop users does not work
> ------------------------------------------------------------
>
>                 Key: HADOOP-16122
>                 URL: https://issues.apache.org/jira/browse/HADOOP-16122
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: auth
>            Reporter: chendihao
>            Priority: Major
>
> In our scenario, we have a service to allow multiple users to access HDFS 
> with their keytab. The users have different Hadoop user and permission to 
> access the HDFS files. The service will run with multi-threads and create one 
> independent UGI object for each user and use the UGI to create Hadoop 
> FileSystem object to read/write HDFS.
>  
> Since we have multiple Hadoop users in the same process, we have to use 
> `loginUserFromKeytabAndReturnUGI` instead of `loginUserFromKeytab`. The 
> `loginUserFromKeytabAndReturnUGI` will not do the re-login automatically. 
> Then we have to call `checkTGTAndReloginFromKeytab` or `reloginFromKeytab` 
> before the kerberos ticket expires.
>  
> The issue is that `reloginFromKeytab` will re-login with the wrong users 
> instead of the one from the expected UGI object.Because of this issue, we can 
> only support multiple Hadoop users to login with their own keytabs but not 
> re-login when the tickets expire.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to