You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-issues@hadoop.apache.org by "Erik.fang (JIRA)" <ji...@apache.org> on 2012/07/17 12:35:34 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] [Commented] (MAPREDUCE-4451) fairscheduler fail to init job with kerberos authentication configured

Posted by "Erik.fang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13463980#comment-13463980 ] 

Erik.fang commented on MAPREDUCE-4451:
--------------------------------------

Thanks for your suggestion, Kihwal

Manually ran test-patch.sh against branch-1
{code}
-1 overall.  

    +1 @author.  The patch does not contain any @author tags.

    -1 tests included.  The patch doesn't appear to include any new or modified tests.
                        Please justify why no tests are needed for this patch.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    -1 findbugs.  The patch appears to introduce 8 new Findbugs (version 1.3.9) warnings.
{code}

- tests: test not include because no behavior changed and no new function added
- findbugs: these warnings is not actually introduced by this patch

                
> 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
>         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

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

Posted by "Karthik Kambatla (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13463007#comment-13463007 ] 

Karthik Kambatla commented on MAPREDUCE-4451:
---------------------------------------------

+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
>         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

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

Posted by "Karthik Kambatla (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13439832#comment-13439832 ] 

Karthik Kambatla commented on MAPREDUCE-4451:
---------------------------------------------

Thanks for the patch, Eric.

I have the following minor comments:
- Use {{jobInitManagerThread}} instead of {{JobInitManagerThread}} (field name starting with lower case)
- In {{initJob()}}, we should probably use {{jobInitQueue.notify()}} instead of {{jobInitQueue.notifyAll()}}. It shouldn't be a problem in the current implementation. Should anyone use more job init threads later, only one of them should be notified.
- In {terminate()}:
-- Log a warning if the {{jobInitQueue}} is not empty.
-- Clear the {{jobInitQueue}}.
-- Not particular on this as it shouldn't affect functionality: we should probably add a {{jobInitManagerThread.join(TIMEOUT)}} in {{terminate()}} for proper clean up.
                
> 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
>
>         Attachments: 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: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (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:comment-tabpanel&focusedCommentId=13474471#comment-13474471 ] 

Alejandro Abdelnur commented on MAPREDUCE-4451:
-----------------------------------------------

+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
>         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

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

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

Erik.fang updated MAPREDUCE-4451:
---------------------------------

    Attachment: MAPREDUCE-4451_branch-1.patch

Thanks Karthik,

new patch attached, add comment and pre-start threads check
                
> 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
>         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

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

Posted by "Karthik Kambatla (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13473495#comment-13473495 ] 

Karthik Kambatla commented on MAPREDUCE-4451:
---------------------------------------------

Thanks Erik. In my opinion, that should be good enough.
                
> 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
>         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

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

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

Erik.fang updated MAPREDUCE-4451:
---------------------------------

    Target Version/s: 1.1.0, 1.1.1
        Release Note: Using FairScheduler with security configured, job initialization fails. The problem is that threads in JobInitializer runs as RPC user instead of jobtracker, pre-start all the threads fix this bug
        Hadoop Flags: Reviewed
              Status: Patch Available  (was: Open)
    
> 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
>         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

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

Posted by "Erik.fang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13473090#comment-13473090 ] 

Erik.fang commented on MAPREDUCE-4451:
--------------------------------------

Before upload the patch ,I have tested it in a 4-node dev-cluster with hadoop-1.0.3 to make sure it works. Maybe it is better to validate the patch with branch-1 compiled jars. I can do it today or tomorrow, and post the result.

However, I can only post some jobtracker logs to show that job initialization fails before apply the patch and every thing works fine after apply the patch. Is that enough or any other ideas?
                
> 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
>         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

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

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

Erik.fang updated MAPREDUCE-4451:
---------------------------------

    Status: Open  (was: Patch Available)
    
> 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
>         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

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

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

Suresh Srinivas updated MAPREDUCE-4451:
---------------------------------------

    Fix Version/s:     (was: 1.1.0)
    
> 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
>         Attachments: 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

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

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

Erik.fang updated MAPREDUCE-4451:
---------------------------------

    Description: 
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.

  was:
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.

    
> 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
>
>         Attachments: 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: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

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

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

Matt Foley updated MAPREDUCE-4451:
----------------------------------

    Target Version/s: 1.2.0  (was: 1.1.0, 1.1.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

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

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

Erik.fang updated MAPREDUCE-4451:
---------------------------------

    Attachment: MAPREDUCE-4451_branch-1.patch

This Patch is for branch-1

This can be fixed by MAPREDUCE-3206, however. we'd better fix the problem introduced by fairscheduler in fairscheduler. I port the code in EagerTaskInitializationListener to seperate job initialization from RPC call path. It works in my cluster.
Any comments or suggestion is appreciated.
                
> 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
>
>         Attachments: 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 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] [Commented] (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:comment-tabpanel&focusedCommentId=13442717#comment-13442717 ] 

Alejandro Abdelnur commented on MAPREDUCE-4451:
-----------------------------------------------

Why not using a ThreadPoolExecutor? I.e.:

{code}
        BlockingQueue<Runnable> queue = new ArrayBlockingQueue<Runnable>(MAX_QUEUE_SIZE);
        executor = new ThreadPoolExecutor(MAX_THEADS, MAX_THREADS, 10, TimeUnit.SECONDS, queue);
{code}

This will get rid of the JobInitMananager innerclass and the notify() calls. in the JobInitializer.

Just be aware of the following trick you'll have to do together with the initialization of the executor.

{code}
        // IMPORTANT: The ThreadPoolExecutor does not always the execute
        // commands out of the queue, there are
        // certain conditions where commands are pushed directly to a thread.
        // As we are using a queue with DELAYED semantics (i.e. execute the
        // command in 5 mins) we need to make
        // sure that the commands are always pushed to the queue.
        // To achieve this (by looking a the ThreadPoolExecutor.execute()
        // implementation, we are making the pool
        // minimum size equals to the maximum size (thus threads are keep always
        // running) and we are warming up
        // all those threads (the for loop that runs dummy runnables).
        for (int i = 0; i < threads; i++) {
            executor.execute(new Runnable() {
                public void run() {
                    try {
                        Thread.sleep(100);
                    }
                    catch (InterruptedException ex) {
                        log.warn("Could not warm up threadpool {0}", ex.getMessage(), ex);
                    }
                }
            });
        }
{code}
                
> 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
>
>         Attachments: 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

[jira] [Commented] (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:comment-tabpanel&focusedCommentId=13469816#comment-13469816 ] 

Alejandro Abdelnur commented on MAPREDUCE-4451:
-----------------------------------------------

Please ignore my last comment, I was wrong there, how you are doing things is correct.
                
> 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
>         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

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

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13463499#comment-13463499 ] 

Hadoop QA commented on MAPREDUCE-4451:
--------------------------------------

{color:red}-1 overall{color}.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12546539/MAPREDUCE-4451_branch-1.patch
  against trunk revision .

    {color:red}-1 patch{color}.  The patch command could not apply the patch.

Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/2881//console

This message is automatically generated.
                
> 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
>         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

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

Posted by "Karthik Kambatla (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13462005#comment-13462005 ] 

Karthik Kambatla commented on MAPREDUCE-4451:
---------------------------------------------

Hi Eric,

Thanks for re-visiting the patch, and sorry I couldn't get to this earlier. 

Nice find on the behavior of {{ThreadPoolExecutor.execute()}} when threads are not pre-started. As the behavior might not be common knowledge, I have the following nits:
# Add a comment before {{threadPool.prestartAllCoreThreads()}} to the effect of - Pre-starting all threads to ensure the threads are executed as JobTracker instead of the user initing the job.
# Add a check {{threadPool.prestartAllCoreThreads() == numThreads}}, and throw an Exception if the check fails - otherwise, it would be hard to debug this

Other than that, the patch looks good to go. Thanks to both Alejandro and you for this informative conversation.


                
> 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
>         Attachments: 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

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

Posted by "Karthik Kambatla (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13474308#comment-13474308 ] 

Karthik Kambatla commented on MAPREDUCE-4451:
---------------------------------------------

+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
>         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

[jira] [Commented] (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:comment-tabpanel&focusedCommentId=13469810#comment-13469810 ] 

Alejandro Abdelnur commented on MAPREDUCE-4451:
-----------------------------------------------

Erik, was not aware of the prestartAllCoreThreads() method, nice.

I think I was not clear explaining the use of ThreadPoolExecutor, it would be something like this:

INITIALIZATION:
  queue = new LinkedBlockingQueue<Runnable> ();
  threadPool = new ThreadPoolExecutor(numThreads, numThreads, 0L, TimeUnit.MILLISECONDS, queue);

INITJOB:
  queue.queue(new InitJob(jobInfo, job));

Then you can queue more InitJobs that threads you have without blocking at INITJOB

Thx
                
> 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
>         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

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

Posted by "Erik.fang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13473956#comment-13473956 ] 

Erik.fang commented on MAPREDUCE-4451:
--------------------------------------

validate the patch with branch-1 revision 1396925

First compile with "ant package", deploy hadoop-core-1.2.0-SNAPSHOT.jar and hadoop-fairscheduler-1.2.0-SNAPSHOT.jar to a 4-node cluster with security on.
run wordcount, job initialization fails. logs from jobtracker
{code} 
2012-10-11 15:50:35,305 INFO org.apache.hadoop.mapred.JobInProgress: job_201210111545_0001: nMaps=5 nReduces=1 max=-1
2012-10-11 15:50:35,425 INFO org.apache.hadoop.mapred.JobTracker: Initializing job_201210111545_0001
2012-10-11 15:50:35,425 INFO org.apache.hadoop.mapred.JobTracker: Job job_201210111545_0001 added successfully for user 'hadoop' to queue 'default'
2012-10-11 15:50:35,425 INFO org.apache.hadoop.mapred.JobInProgress: Initializing job_201210111545_0001
2012-10-11 15:50:35,427 INFO org.apache.hadoop.mapred.AuditLogger: USER=hadoop	IP=192.168.7.80	OPERATION=SUBMIT_JOB	TARGET=job_201210111545_0001	RESULT=SUCCESS
2012-10-11 15:50:35,601 ERROR org.apache.hadoop.security.UserGroupInformation: PriviledgedActionException as:hadoop@DIANPING.COM cause:javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
2012-10-11 15:50:35,602 WARN org.apache.hadoop.ipc.Client: Exception encountered while connecting to the server : javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
2012-10-11 15:50:35,602 ERROR org.apache.hadoop.security.UserGroupInformation: PriviledgedActionException as:hadoop@DIANPING.COM cause: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)]
2012-10-11 15:50:35,603 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:266)
	at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:231)
	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:3556)
	at org.apache.hadoop.mapred.JobInProgress.initTasks(JobInProgress.java:696)
	at org.apache.hadoop.mapred.JobTracker.initJob(JobTracker.java:4028)
	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

2012-10-11 15:50:35,603 INFO org.apache.hadoop.mapred.JobTracker: Failing job job_201210111545_0001
2012-10-11 15:50:35,609 INFO org.apache.hadoop.mapred.JobInProgress$JobSummary: jobId=job_201210111545_0001,submitTime=1349941835229,launchTime=0,,finishTime=1349941835603,numMaps=0,numSlotsPerMap=1,numReduces=0,numSlotsPerReduce=1,user=hadoop,queue=default,status=FAILED,mapSlotSeconds=0,reduceSlotsSeconds=0,clusterMapCapacity=6,clusterReduceCapacity=6,jobName=word count
2012-10-11 15:50:35,633 INFO org.apache.hadoop.mapred.JobHistory: Creating DONE subfolder at file:/data/logs/history/done/version-1/dev80.hadoop_1349941538601_/2012/10/11/000000
2012-10-11 15:50:35,634 INFO org.apache.hadoop.mapred.JobHistory: Moving file:/data/logs/history/job_201210111545_0001_1349941835229_hadoop_word+count to file:/data/logs/history/done/version-1/dev80.hadoop_1349941538601_/2012/10/11/000000
2012-10-11 15:50:35,638 INFO org.apache.hadoop.hdfs.DFSClient: Cancelling HDFS_DELEGATION_TOKEN token 459 for hadoop on 192.168.7.80:8020
2012-10-11 15:50:35,642 INFO org.apache.hadoop.mapred.JobHistory: Moving file:/data/logs/history/job_201210111545_0001_conf.xml to file:/data/logs/history/done/version-1/dev80.hadoop_1349941538601_/2012/10/11/000000

{code}

apply the patch with "patch -p0 < MAPREDUCE-4451_branch-1.patch"
compile with "ant clean""ant package", deploy new hadoop-fairscheduler-1.2.0-SNAPSHOT.jar, and run wordcount. Everything works fine.
{code}
2012-10-11 16:10:40,253 INFO org.apache.hadoop.mapred.JobInProgress: job_201210111609_0001: nMaps=5 nReduces=1 max=-1
2012-10-11 16:10:40,254 INFO org.apache.hadoop.hdfs.DFSClient: Renewing HDFS_DELEGATION_TOKEN token 460 for hadoop on 192.168.7.80:8020
2012-10-11 16:10:40,379 INFO org.apache.hadoop.mapred.JobTracker: Initializing job_201210111609_0001
2012-10-11 16:10:40,379 INFO org.apache.hadoop.mapred.JobInProgress: Initializing job_201210111609_0001
2012-10-11 16:10:40,379 INFO org.apache.hadoop.mapred.JobTracker: Job job_201210111609_0001 added successfully for user 'hadoop' to queue 'default'
2012-10-11 16:10:40,382 INFO org.apache.hadoop.mapred.AuditLogger: USER=hadoop	IP=192.168.7.80	OPERATION=SUBMIT_JOB	TARGET=job_201210111609_0001	RESULT=SUCCESS
2012-10-11 16:10:40,524 INFO org.apache.hadoop.mapred.JobInProgress: jobToken generated and stored with users keys in /data/mapred/system/job_201210111609_0001/jobToken
2012-10-11 16:10:40,540 INFO org.apache.hadoop.mapred.JobInProgress: Input size for job job_201210111609_0001 = 75545. Number of splits = 5
2012-10-11 16:10:40,540 INFO org.apache.hadoop.mapred.JobInProgress: tip:task_201210111609_0001_m_000000 has split on node:/rack2/dev82.hadoop
2012-10-11 16:10:40,540 INFO org.apache.hadoop.mapred.JobInProgress: tip:task_201210111609_0001_m_000000 has split on node:/rack2/dev83.hadoop
2012-10-11 16:10:40,541 INFO org.apache.hadoop.mapred.JobInProgress: tip:task_201210111609_0001_m_000001 has split on node:/rack2/dev83.hadoop
2012-10-11 16:10:40,541 INFO org.apache.hadoop.mapred.JobInProgress: tip:task_201210111609_0001_m_000001 has split on node:/rack2/dev82.hadoop
2012-10-11 16:10:40,541 INFO org.apache.hadoop.mapred.JobInProgress: tip:task_201210111609_0001_m_000002 has split on node:/rack2/dev83.hadoop
2012-10-11 16:10:40,541 INFO org.apache.hadoop.mapred.JobInProgress: tip:task_201210111609_0001_m_000002 has split on node:/rack2/dev82.hadoop
2012-10-11 16:10:40,541 INFO org.apache.hadoop.mapred.JobInProgress: tip:task_201210111609_0001_m_000003 has split on node:/rack2/dev83.hadoop
2012-10-11 16:10:40,542 INFO org.apache.hadoop.mapred.JobInProgress: tip:task_201210111609_0001_m_000003 has split on node:/rack2/dev82.hadoop
2012-10-11 16:10:40,542 INFO org.apache.hadoop.mapred.JobInProgress: tip:task_201210111609_0001_m_000004 has split on node:/rack2/dev83.hadoop
2012-10-11 16:10:40,542 INFO org.apache.hadoop.mapred.JobInProgress: tip:task_201210111609_0001_m_000004 has split on node:/rack2/dev82.hadoop
2012-10-11 16:10:40,542 INFO org.apache.hadoop.mapred.JobInProgress: job_201210111609_0001 LOCALITY_WAIT_FACTOR=0.6666667
2012-10-11 16:10:40,543 INFO org.apache.hadoop.mapred.JobInProgress: Job job_201210111609_0001 initialized successfully with 5 map tasks and 1 reduce tasks.
2012-10-11 16:10:40,704 INFO org.apache.hadoop.mapred.JobTracker: Adding task (JOB_SETUP) 'attempt_201210111609_0001_m_000006_0' to tip task_201210111609_0001_m_000006, for tracker 'tracker_dev81.hadoop:localhost/127.0.0.1:55568'
2012-10-11 16:10:43,157 INFO org.apache.hadoop.mapred.JobInProgress: Task 'attempt_201210111609_0001_m_000006_0' has completed task_201210111609_0001_m_000006 successfully.
2012-10-11 16:10:43,730 INFO org.apache.hadoop.mapred.JobTracker: Adding task (MAP) 'attempt_201210111609_0001_m_000000_0' to tip task_201210111609_0001_m_000000, for tracker 'tracker_dev82.hadoop:localhost/127.0.0.1:59770'
2012-10-11 16:10:43,731 INFO org.apache.hadoop.mapred.JobInProgress: Choosing data-local task task_201210111609_0001_m_000000
2012-10-11 16:10:43,731 INFO org.apache.hadoop.mapred.JobTracker: Adding task (MAP) 'attempt_201210111609_0001_m_000001_0' to tip task_201210111609_0001_m_000001, for tracker 'tracker_dev82.hadoop:localhost/127.0.0.1:59770'
2012-10-11 16:10:43,732 INFO org.apache.hadoop.mapred.JobInProgress: Choosing data-local task task_201210111609_0001_m_000001
2012-10-11 16:10:43,732 INFO org.apache.hadoop.mapred.JobTracker: Adding task (MAP) 'attempt_201210111609_0001_m_000002_0' to tip task_201210111609_0001_m_000002, for tracker 'tracker_dev83.hadoop:localhost/127.0.0.1:60713'
2012-10-11 16:10:43,733 INFO org.apache.hadoop.mapred.JobInProgress: Choosing data-local task task_201210111609_0001_m_000002
2012-10-11 16:10:43,733 INFO org.apache.hadoop.mapred.JobTracker: Adding task (MAP) 'attempt_201210111609_0001_m_000003_0' to tip task_201210111609_0001_m_000003, for tracker 'tracker_dev83.hadoop:localhost/127.0.0.1:60713'
2012-10-11 16:10:43,733 INFO org.apache.hadoop.mapred.JobInProgress: Choosing data-local task task_201210111609_0001_m_000003
2012-10-11 16:10:44,975 INFO org.apache.hadoop.mapred.JobInProgress: Choosing a non-local task task_201210111609_0001_m_000004
2012-10-11 16:10:44,975 INFO org.apache.hadoop.mapred.JobTracker: Adding task (MAP) 'attempt_201210111609_0001_m_000004_0' to tip task_201210111609_0001_m_000004, for tracker 'tracker_dev81.hadoop:localhost/127.0.0.1:55568'
2012-10-11 16:10:46,486 INFO org.apache.hadoop.mapred.JobInProgress: Task 'attempt_201210111609_0001_m_000000_0' has completed task_201210111609_0001_m_000000 successfully.
2012-10-11 16:10:46,489 INFO org.apache.hadoop.mapred.JobInProgress: Task 'attempt_201210111609_0001_m_000003_0' has completed task_201210111609_0001_m_000003 successfully.
2012-10-11 16:10:46,494 INFO org.apache.hadoop.mapred.JobInProgress: Task 'attempt_201210111609_0001_m_000004_0' has completed task_201210111609_0001_m_000004 successfully.
2012-10-11 16:10:46,792 INFO org.apache.hadoop.mapred.JobInProgress: Task 'attempt_201210111609_0001_m_000001_0' has completed task_201210111609_0001_m_000001 successfully.
2012-10-11 16:10:46,798 INFO org.apache.hadoop.mapred.JobTracker: Adding task (REDUCE) 'attempt_201210111609_0001_r_000000_0' to tip task_201210111609_0001_r_000000, for tracker 'tracker_dev82.hadoop:localhost/127.0.0.1:59770'
2012-10-11 16:10:46,799 INFO org.apache.hadoop.mapred.JobInProgress: Task 'attempt_201210111609_0001_m_000002_0' has completed task_201210111609_0001_m_000002 successfully.
2012-10-11 16:10:55,568 INFO org.apache.hadoop.mapred.JobInProgress: Task 'attempt_201210111609_0001_r_000000_0' has completed task_201210111609_0001_r_000000 successfully.
2012-10-11 16:10:55,573 INFO org.apache.hadoop.mapred.JobTracker: Adding task (JOB_CLEANUP) 'attempt_201210111609_0001_m_000005_0' to tip task_201210111609_0001_m_000005, for tracker 'tracker_dev82.hadoop:localhost/127.0.0.1:59770'
2012-10-11 16:10:56,789 INFO org.apache.hadoop.mapred.JobInProgress: Task 'attempt_201210111609_0001_m_000005_0' has completed task_201210111609_0001_m_000005 successfully.
2012-10-11 16:10:56,790 INFO org.apache.hadoop.mapred.JobInProgress: Job job_201210111609_0001 has completed successfully.
2012-10-11 16:10:56,793 INFO org.apache.hadoop.mapred.JobInProgress$JobSummary: jobId=job_201210111609_0001,submitTime=1349943040191,launchTime=1349943040542,firstMapTaskLaunchTime=1349943043730,firstReduceTaskLaunchTime=1349943046794,firstJobSetupTaskLaunchTime=1349943040684,firstJobCleanupTaskLaunchTime=1349943055573,finishTime=1349943056790,numMaps=5,numSlotsPerMap=1,numReduces=1,numSlotsPerReduce=1,user=hadoop,queue=default,status=SUCCEEDED,mapSlotSeconds=9,reduceSlotsSeconds=8,clusterMapCapacity=6,clusterReduceCapacity=6,jobName=word count
2012-10-11 16:10:56,822 INFO org.apache.hadoop.mapred.JobHistory: Creating DONE subfolder at file:/data/logs/history/done/version-1/dev80.hadoop_1349942948793_/2012/10/11/000000
2012-10-11 16:10:56,824 INFO org.apache.hadoop.mapred.JobHistory: Moving file:/data/logs/history/job_201210111609_0001_1349943040191_hadoop_word+count to file:/data/logs/history/done/version-1/dev80.hadoop_1349942948793_/2012/10/11/000000
2012-10-11 16:10:56,825 INFO org.apache.hadoop.mapred.JobTracker: Removing task 'attempt_201210111609_0001_m_000000_0'
2012-10-11 16:10:56,833 INFO org.apache.hadoop.mapred.JobHistory: Moving file:/data/logs/history/job_201210111609_0001_conf.xml to file:/data/logs/history/done/version-1/dev80.hadoop_1349942948793_/2012/10/11/000000
2012-10-11 16:10:56,827 INFO org.apache.hadoop.hdfs.DFSClient: Cancelling HDFS_DELEGATION_TOKEN token 460 for hadoop on 192.168.7.80:8020
2012-10-11 16:10:56,835 INFO org.apache.hadoop.mapred.JobTracker: Removing task 'attempt_201210111609_0001_m_000001_0'
2012-10-11 16:10:56,835 INFO org.apache.hadoop.mapred.JobTracker: Removing task 'attempt_201210111609_0001_m_000005_0'
2012-10-11 16:10:56,835 INFO org.apache.hadoop.mapred.JobTracker: Removing task 'attempt_201210111609_0001_r_000000_0'
2012-10-11 16:10:56,897 WARN org.apache.hadoop.mapred.JobHistory: JobHistory: existingDoneSubdirs doesn't contain file:/data/logs/history/done/version-1/dev80.hadoop_1346294197416_/2012/08/30/000000, but should.
2012-10-11 16:10:56,899 WARN org.apache.hadoop.mapred.JobHistory: JobHistory: existingDoneSubdirs doesn't contain file:/data/logs/history/done/version-1/dev80.hadoop_1346351022278_/2012/08/31/000000, but should.
2012-10-11 16:10:56,905 WARN org.apache.hadoop.mapred.JobHistory: JobHistory: existingDoneSubdirs doesn't contain file:/data/logs/history/done/version-1/dev80.hadoop_1346385661102_/2012/09/03/000000, but should.
2012-10-11 16:10:56,908 WARN org.apache.hadoop.mapred.JobHistory: JobHistory: existingDoneSubdirs doesn't contain file:/data/logs/history/done/version-1/dev80.hadoop_1346385661102_/2012/08/31/000000, but should.
2012-10-11 16:10:56,909 WARN org.apache.hadoop.mapred.JobHistory: JobHistory: existingDoneSubdirs doesn't contain file:/data/logs/history/done/version-1/dev80.hadoop_1346254764307_/2012/08/29/000000, but should.
2012-10-11 16:10:56,911 WARN org.apache.hadoop.mapred.JobHistory: JobHistory: existingDoneSubdirs doesn't contain file:/data/logs/history/done/version-1/dev80.hadoop_1346350655601_/2012/08/31/000000, but should.
2012-10-11 16:10:57,070 INFO org.apache.hadoop.mapred.JobTracker: Removing task 'attempt_201210111609_0001_m_000002_0'
2012-10-11 16:10:57,070 INFO org.apache.hadoop.mapred.JobTracker: Removing task 'attempt_201210111609_0001_m_000003_0'
2012-10-11 16:10:57,071 INFO org.apache.hadoop.mapred.JobTracker: Removing task 'attempt_201210111609_0001_m_000004_0'
2012-10-11 16:10:57,071 INFO org.apache.hadoop.mapred.JobTracker: Removing task 'attempt_201210111609_0001_m_000006_0'

{code}
                
> 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
>         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

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

Posted by "Karthik Kambatla (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13473076#comment-13473076 ] 

Karthik Kambatla commented on MAPREDUCE-4451:
---------------------------------------------

Given that the patch doesn't have tests, I was planning on running a secure cluster to ascertain the behavior, but haven't been able to get to it.

Will you be able to validate the behavior and report the same? Otherwise, I ll see if I can do the same in the next couple of days?
                
> 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
>         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

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

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

Erik.fang updated MAPREDUCE-4451:
---------------------------------

    Attachment: MAPREDUCE-4451_branch-1.patch

Hi Alejandro,
Using a Blocking Queue and ThreadPoolExecutor can do the same job, while I mostly ported the codes in EagerTaskInitializationListener used by early version of fairscheduler.

Inspired by your comment, I look at the implementation of ThreadPoolExecutor.execute(), and find the root cause.
The main problem is where the threads in ThreadPoolExecutor created. When a new job arrives, threadPool.execute(new InitJob(jobInfo, job)) is called. If there is less than corePoolSize threads in the threadPool, a new thread is created ( in ThreadPoolExecutor.addIfUnderCorePoolSize() ), which is in the UGI.doAs() block as the RPC client remote user.
So we just warm up all the threads in the pool, using threadPool.prestartAllCoreThreads().

This can be easily demonstrated:
{code} 
public class testthreadpool {
	ThreadPoolExecutor threadPool;
	UserGroupInformation u1;
	UserGroupInformation u2;

	class task implements Runnable {
		@Override
		public void run() {
			try {
				UserGroupInformation u = UserGroupInformation.getCurrentUser();
				System.out.println(u.getUserName());
			} catch (IOException e) {
				e.printStackTrace();
			}
		}
	}
	public void init() throws IOException {	
		u1 = UserGroupInformation.getCurrentUser();
		u2= UserGroupInformation.createRemoteUser("tony");	
		threadPool = (ThreadPoolExecutor) Executors.newFixedThreadPool(1);
		threadPool.prestartAllCoreThreads();
	}
	public void dowork() throws IOException, InterruptedException {
		u2.doAs(new PrivilegedExceptionAction<Object>() {
			public Object run() throws Exception {
				threadPool.execute(new task());
				return null;
			}			
		});
	}
	public void done() {
		threadPool.shutdown();
	}
	public static void main(String[] args) throws IOException, InterruptedException {
		testthreadpool t = new testthreadpool();
		t.init();
		t.dowork();
		t.done();
	}
}
{code} 
the result is "erik"
comment threadPool.prestartAllCoreThreads(), and we got "tony"

new Patch uploaded

Thanks,
Erik
                
> 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
>
>         Attachments: 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

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

Posted by "Kihwal Lee (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13463881#comment-13463881 ] 

Kihwal Lee commented on MAPREDUCE-4451:
---------------------------------------

Erik, you can run src/test/bin/test-patch.sh manually and post the result.
                
> 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
>         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

[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

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

Posted by "Karthik Kambatla (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13463997#comment-13463997 ] 

Karthik Kambatla commented on MAPREDUCE-4451:
---------------------------------------------

bq. tests: test not include because no behavior changed and no new function added

I think the behavior has indeed changed, from wrong to right. A test in this case would prevent relapsing to the old behavior. However, I understand writing a test might prove to be hard in this case.
                
> 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
>         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

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

Posted by "Karthik Kambatla (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-4451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13442523#comment-13442523 ] 

Karthik Kambatla commented on MAPREDUCE-4451:
---------------------------------------------

Looks good to me.
                
> 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
>
>         Attachments: 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