You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-dev@hadoop.apache.org by "Erik.fang (JIRA)" <ji...@apache.org> on 2012/07/17 12:35:33 UTC

[jira] [Created] (MAPREDUCE-4451) fairscheduler fail to init job with kerberos authentication configured

Erik.fang created MAPREDUCE-4451:
------------------------------------

             Summary: fairscheduler fail to init job with kerberos authentication configured
                 Key: MAPREDUCE-4451
                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-4451
             Project: Hadoop Map/Reduce
          Issue Type: Bug
          Components: contrib/fair-share
    Affects Versions: 1.0.3
            Reporter: Erik.fang
             Fix For: 1.1.0


Using FairScheduler in Hadoop 1.0.3 with kerberos authentication configured. Job initialization fails:

{code}
2012-07-17 15:15:09,220 ERROR org.apache.hadoop.mapred.JobTracker: Job initialization failed:
java.io.IOException: Call to /192.168.7.80:8020 failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
        at org.apache.hadoop.ipc.Client.wrapException(Client.java:1129)
        at org.apache.hadoop.ipc.Client.call(Client.java:1097)
        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:229)
        at $Proxy7.getProtocolVersion(Unknown Source)
        at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:411)
        at org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:125)
        at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:329)
        at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:294)
        at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:100)
        at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1411)
        at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:66)
        at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1429)
        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:254)
        at org.apache.hadoop.fs.Path.getFileSystem(Path.java:187)
        at org.apache.hadoop.security.Credentials.writeTokenStorageFile(Credentials.java:169)
        at org.apache.hadoop.mapred.JobInProgress.generateAndStoreTokens(JobInProgress.java:3558)
        at org.apache.hadoop.mapred.JobInProgress.initTasks(JobInProgress.java:696)
        at org.apache.hadoop.mapred.JobTracker.initJob(JobTracker.java:3911)
        at org.apache.hadoop.mapred.FairScheduler$JobInitializer$InitJob.run(FairScheduler.java:301)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
Caused by: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
        at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:543)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:396)
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1136)
        at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:488)
        at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:590)
        at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:187)
        at org.apache.hadoop.ipc.Client.getConnection(Client.java:1228)
        at org.apache.hadoop.ipc.Client.call(Client.java:1072)
        ... 20 more
Caused by: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
        at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:194)
        at org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:134)
        at org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:385)
        at org.apache.hadoop.ipc.Client$Connection.access$1200(Client.java:187)
        at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:583)
        at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:580)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:396)
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1136)
        at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:579)
        ... 23 more
Caused by: GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)
        at sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:130)
        at sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:106)
        at sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:172)
        at sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:209)
        at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:195)
        at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:162)
        at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:175)
        ... 32 more

{code}

When a job is submited, fairscheduler calls JobTracker.initJob, which calls JobInProgress.generateAndStoreTokens to write security keys to hdfs. However, the operation is involved in the server side rpc call path, using UGI created by UserGroupInformation.createRemoteUser in rpc server, which have no tgt. This should be done with UGI used by JobTracker.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Resolved] (MAPREDUCE-4451) fairscheduler fail to init job with kerberos authentication configured

Posted by "Alejandro Abdelnur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Alejandro Abdelnur resolved MAPREDUCE-4451.
-------------------------------------------

       Resolution: Fixed
    Fix Version/s: 1.2.0
         Assignee: Erik.fang

Thanks Erik. Committed to branch-1.
                
> fairscheduler fail to init job with kerberos authentication configured
> ----------------------------------------------------------------------
>
>                 Key: MAPREDUCE-4451
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-4451
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: contrib/fair-share
>    Affects Versions: 1.0.3
>            Reporter: Erik.fang
>            Assignee: Erik.fang
>             Fix For: 1.2.0
>
>         Attachments: MAPREDUCE-4451_branch-1.patch, MAPREDUCE-4451_branch-1.patch, MAPREDUCE-4451_branch-1.patch, MAPREDUCE-4451_branch-1.patch, MAPREDUCE-4451_branch-1.patch
>
>
> Using FairScheduler in Hadoop 1.0.3 with kerberos authentication configured. Job initialization fails:
> {code}
> 2012-07-17 15:15:09,220 ERROR org.apache.hadoop.mapred.JobTracker: Job initialization failed:
> java.io.IOException: Call to /192.168.7.80:8020 failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
>         at org.apache.hadoop.ipc.Client.wrapException(Client.java:1129)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1097)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:229)
>         at $Proxy7.getProtocolVersion(Unknown Source)
>         at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:411)
>         at org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:125)
>         at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:329)
>         at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:294)
>         at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:100)
>         at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1411)
>         at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:66)
>         at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1429)
>         at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:254)
>         at org.apache.hadoop.fs.Path.getFileSystem(Path.java:187)
>         at org.apache.hadoop.security.Credentials.writeTokenStorageFile(Credentials.java:169)
>         at org.apache.hadoop.mapred.JobInProgress.generateAndStoreTokens(JobInProgress.java:3558)
>         at org.apache.hadoop.mapred.JobInProgress.initTasks(JobInProgress.java:696)
>         at org.apache.hadoop.mapred.JobTracker.initJob(JobTracker.java:3911)
>         at org.apache.hadoop.mapred.FairScheduler$JobInitializer$InitJob.run(FairScheduler.java:301)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>         at java.lang.Thread.run(Thread.java:662)
> Caused by: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
>         at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:543)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at javax.security.auth.Subject.doAs(Subject.java:396)
>         at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1136)
>         at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:488)
>         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:590)
>         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:187)
>         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1228)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1072)
>         ... 20 more
> Caused by: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
>         at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:194)
>         at org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:134)
>         at org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:385)
>         at org.apache.hadoop.ipc.Client$Connection.access$1200(Client.java:187)
>         at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:583)
>         at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:580)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at javax.security.auth.Subject.doAs(Subject.java:396)
>         at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1136)
>         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:579)
>         ... 23 more
> Caused by: GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)
>         at sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:130)
>         at sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:106)
>         at sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:172)
>         at sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:209)
>         at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:195)
>         at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:162)
>         at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:175)
>         ... 32 more
> {code}
> When a job is submitted, fairscheduler calls JobTracker.initJob, which calls JobInProgress.generateAndStoreTokens to write security keys to hdfs. However, the operation is involved in the server side rpc call path, using UGI created by UserGroupInformation.createRemoteUser in rpc server, which have no tgt. This should be done with UGI used by JobTracker.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira