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

Istvan Fajth commented on HDFS-14668:
-------------------------------------

After an extensive debugging session, I was able to come up with a Java code 
that reproduces the problem, and was able to nail down the problem as I think, 
and this is broken because of two things, one is the API with which FUSE and 
the native HDFS client connects to the FileSystem API, and how that uses 
UserGroupInformation, and a change in the default behaviour of 
UserGroupInformation in HADOOP-9747.

The Java code reproduces the issue:
{code}
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;

import java.io.IOException;
import java.net.URI;

public class TestUGITheFUSEWay {
  public static void main(String[] args)
      throws IOException, InterruptedException {

    Configuration conf = new Configuration();

    URI ns = URI.create("dfs://ns1");
    String krbTicketCachePath = "/tmp/krb5cc_1000";
    conf.set("hadoop.security.kerberos.ticket.cache.path", krbTicketCachePath);
    String userName = "test";

    FileSystem fs = FileSystem.newInstance(ns, conf, userName);
    Path p= new Path("/");

    for (FileStatus s : fs.listStatus(p)){
      System.out.println(s.toString());
    }
  }
}
{code}

The underlying problem is with the fact that in case you specify the userName 
to the FileSystem.newInstance call, then in the old word UserGroupInformation 
did not added the userName to the Kerberos configuration entries as the 
principal, so that as the unerlying Java libraries used the principal in the 
ticket cache as the principal.
After HADOOP-9747 if you specify the userName there, it is added to the 
AppConfigurationEntry given to the HadoopLoginContext in the 
HadoopLoginConfiguration as principal, and then the underlying Java libraries 
will throw an indicative exception (if kerberos debugging is on):
{code}
Acquire TGT from Cache
java.io.IOException: Primary principals don't match.
        at 
sun.security.krb5.internal.ccache.FileCredentialsCache.load(FileCredentialsCache.java:179)
        at 
sun.security.krb5.internal.ccache.FileCredentialsCache.acquireInstance(FileCredentialsCache.java:82)
        at 
sun.security.krb5.internal.ccache.CredentialsCache.getInstance(CredentialsCache.java:83)
        at 
sun.security.krb5.Credentials.acquireTGTFromCache(Credentials.java:333)
        at 
com.sun.security.auth.module.Krb5LoginModule.attemptAuthentication(Krb5LoginModule.java:665)
        at 
com.sun.security.auth.module.Krb5LoginModule.login(Krb5LoginModule.java:617)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at javax.security.auth.login.LoginContext.invoke(LoginContext.java:755)
        at 
javax.security.auth.login.LoginContext.access$000(LoginContext.java:195)
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:682)
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:680)
        at java.security.AccessController.doPrivileged(Native Method)
        at 
javax.security.auth.login.LoginContext.invokePriv(LoginContext.java:680)
        at javax.security.auth.login.LoginContext.login(LoginContext.java:587)
        at 
org.apache.hadoop.security.UserGroupInformation$HadoopLoginContext.login(UserGroupInformation.java:2086)
        at 
org.apache.hadoop.security.UserGroupInformation.doSubjectLogin(UserGroupInformation.java:1993)
        at 
org.apache.hadoop.security.UserGroupInformation.getUGIFromTicketCache(UserGroupInformation.java:654)
        at 
org.apache.hadoop.security.UserGroupInformation.getBestUGI(UserGroupInformation.java:624)
        at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:495)
{code}
And ultimately it will fail with:
{code}
LoginException: Unable to obtain password from user
org.apache.hadoop.security.KerberosAuthException: failure to login: for 
principal: systest using ticket cache file: /tmp/krb5cc_2000 
javax.security.auth.login.LoginException: Unable to obtain password from user

        at 
org.apache.hadoop.security.UserGroupInformation.doSubjectLogin(UserGroupInformation.java:2008)
        at 
org.apache.hadoop.security.UserGroupInformation.getUGIFromTicketCache(UserGroupInformation.java:654)
        at 
org.apache.hadoop.security.UserGroupInformation.getBestUGI(UserGroupInformation.java:624)
        at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:495)
Caused by: javax.security.auth.login.LoginException: Unable to obtain password 
from user

        at 
com.sun.security.auth.module.Krb5LoginModule.promptForPass(Krb5LoginModule.java:897)
        at 
com.sun.security.auth.module.Krb5LoginModule.attemptAuthentication(Krb5LoginModule.java:760)
        at 
com.sun.security.auth.module.Krb5LoginModule.login(Krb5LoginModule.java:617)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at javax.security.auth.login.LoginContext.invoke(LoginContext.java:755)
        at 
javax.security.auth.login.LoginContext.access$000(LoginContext.java:195)
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:682)
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:680)
        at java.security.AccessController.doPrivileged(Native Method)
        at 
javax.security.auth.login.LoginContext.invokePriv(LoginContext.java:680)
        at javax.security.auth.login.LoginContext.login(LoginContext.java:587)
        at 
org.apache.hadoop.security.UserGroupInformation$HadoopLoginContext.login(UserGroupInformation.java:2086)
        at 
org.apache.hadoop.security.UserGroupInformation.doSubjectLogin(UserGroupInformation.java:1993)
        ... 3 more
{code}
As it falls back to trying to ask for a password for the user which it can not 
do.

There can be 4 cases in the fuse use case where you have multiple realms with 
users sometime different from the actual OS user running something against FUSE 
with a principal:
# the username of the OS user and the principal match, and the principal is in 
the default realm
# the username of the OS user is different from the principal, and the 
principal is in the default realm
# the username of the OS user and the principal match, but the principal is not 
in the default realm
# the username pf the OS user is different from the principal, and the 
principal is not in the default realm

>From the above #1 is working, none of the others work.

Possible solutions can be:
- either provide null as the username from FUSE, but this solves just the FUSE 
problem, and via libHDFS there is a builder that expects a username, which 
might be set in other workflows, this does not seem to be the best.
- or leave out the specification of the provided userName from the 
AppConfigurationEntry's options provided to the unerlying kerberos library.

[~daryn] as you have created the original patch, and did most of the work, I 
would really love to get some thoughts for you on this, whether there is a 
reason to provide the userName as a principal or it did just sound a good thing 
to do there. And also it would be nice to discuss the possible solutions if you 
have some time.

CC:[~bharat] as the one who reviewed the original patch

> Support Fuse with Users from multiple Security Realms
> -----------------------------------------------------
>
>                 Key: HDFS-14668
>                 URL: https://issues.apache.org/jira/browse/HDFS-14668
>             Project: Hadoop HDFS
>          Issue Type: Improvement
>          Components: fuse-dfs
>            Reporter: Sailesh Patel
>            Assignee: Istvan Fajth
>            Priority: Minor
>
> Users from non-default  krb5 domain can't use hadoop-fuse.
> There are 2 Realms with kdc. 
>     -one realm is for human users  (USERS.COM.US) 
>     -the other is for service principals.   (SERVICE.COM.US) 
> Cross realm trust is setup.
> In krb5.conf  the default domain  is set to SERVICE.COM.US
> Users within USERS.COM.US Realm are not able to put any files to Fuse mounted 
> location
> The client shows:
>       cp: cannot create regular file ‘/hdfs_mount/tmp/hello_from_fuse.txt’: 
> Input/output error



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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

Reply via email to