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 "Gary Helmling (Created) (JIRA)" <ji...@apache.org> on 2011/10/19 03:21:10 UTC
[jira] [Created] (MAPREDUCE-3206) FairScheduler breaks writing of
job tokens to MR system dir when using kerberos auth
FairScheduler breaks writing of job tokens to MR system dir when using kerberos auth
------------------------------------------------------------------------------------
Key: MAPREDUCE-3206
URL: https://issues.apache.org/jira/browse/MAPREDUCE-3206
Project: Hadoop Map/Reduce
Issue Type: Bug
Components: contrib/fair-share
Affects Versions: 0.20.205.0
Reporter: Gary Helmling
Priority: Critical
Using the FairScheduler with kerberos authentication does not appear to work with 0.20.205.0. When submitting a job, execution fails when storing the delegation tokens for the job:
{noformat}
11/10/18 21:48:53 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 6 for hdfs on xxxx:8020
11/10/18 21:48:53 INFO security.TokenCache: Got dt for hdfs://xxxx/user/hdfs/.staging/job_201110182148_0001;uri=xxxx:8020;t.service=xxxx:8020
11/10/18 21:48:53 INFO mapred.FileInputFormat: Total input paths to process : 1
11/10/18 21:48:53 INFO mapred.JobClient: Running job: job_201110182148_0001
11/10/18 21:48:54 INFO mapred.JobClient: map 0% reduce 0%
11/10/18 21:48:54 INFO mapred.JobClient: Job complete: job_201110182148_0001
11/10/18 21:48:54 INFO mapred.JobClient: Counters: 0
11/10/18 21:48:54 INFO mapred.JobClient: Job Failed: Job initialization failed:
java.io.IOException: Call to xxxx/xxxx:8020 failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
at org.apache.hadoop.ipc.Client.wrapException(Client.java:1103)
at org.apache.hadoop.ipc.Client.call(Client.java:1071)
at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
at $Proxy7.getProtocolVersion(Unknown Source)
at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:396)
at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:379)
at org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:118)
at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:222)
at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:187)
at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:89)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1328)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:65)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1346)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:244)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:187)
at org.apache.hadoop.security.Credentials.writeTokenStorageFile(Credentials.java:175)
at org.apache.hadoop.mapred.JobInProgress.generateAndStoreTokens(JobInProgress.java:3528)
at org.apache.hadoop.mapred.JobInProgress.initTasks(JobInProgress.java:696)
at org.apache.hadoop.mapred.JobTracker.initJob(JobTracker.java:4207)
at org.apache.hadoop.mapred.FairScheduler$JobInitializer$InitJob.run(FairScheduler.java:291)
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: Attempt to obtain new INITIATE credentials failed! (null))]
at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:539)
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:1059)
at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:484)
at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:586)
at org.apache.hadoop.ipc.Client$Connection.access$2000(Client.java:184)
at org.apache.hadoop.ipc.Client.getConnection(Client.java:1202)
at org.apache.hadoop.ipc.Client.call(Client.java:1046)
... 21 more
Caused by: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
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:381)
at org.apache.hadoop.ipc.Client$Connection.access$1100(Client.java:184)
at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:579)
at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:576)
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:1059)
at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:575)
... 24 more
Caused by: GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))
at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:333)
at sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:128)
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)
... 33 more
Caused by: javax.security.auth.login.LoginException: No LoginModules configured for
at javax.security.auth.login.LoginContext.init(LoginContext.java:256)
at javax.security.auth.login.LoginContext.<init>(LoginContext.java:499)
at sun.security.jgss.GSSUtil.login(GSSUtil.java:244)
at sun.security.jgss.krb5.Krb5Util.getTicket(Krb5Util.java:136)
at sun.security.jgss.krb5.Krb5InitCredential$1.run(Krb5InitCredential.java:328)
at java.security.AccessController.doPrivileged(Native Method)
at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:325)
... 40 more
{noformat}
The problem seems to have been introduced by the backported changes in MAPREDUCE-2981, which shifted the execution of JobTracker.initJob(), and hence JobInProgress.generateAndStoreTokens(), to underneath the call path for the RPC invocation. As a result, the DFS write in TokenStorage.writeTokenStorageFile() in done under a UGI.doAs() block as the RPC client remote user, without a TGT for negotiating the connection.
Does this analysis seem right? Previously it seems that JobTracker.initJob() was only called in a separate thread so it was picking up the credentials obtained for the configured JobTracker kerberos principal. The same job runs successfully in a build with MAPREDUCE-2981 reverted.
--
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-3206) FairScheduler breaks writing of
job tokens to MR system dir when using kerberos auth
Posted by "Gary Helmling (Commented) (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/MAPREDUCE-3206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13134652#comment-13134652 ]
Gary Helmling commented on MAPREDUCE-3206:
------------------------------------------
>From looking at the current code in branch-0.23, it seems that version of the fair scheduler code has the same problem. But, according to MAPREDUCE-2736, the fair scheduler contrib is slated for removal in 0.23. So I don't know if there is any point in attempting a more extensive refactoring of the fair scheduler code to avoid the problem, as opposed to the simpler fix that I posted.
> FairScheduler breaks writing of job tokens to MR system dir when using kerberos auth
> ------------------------------------------------------------------------------------
>
> Key: MAPREDUCE-3206
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-3206
> Project: Hadoop Map/Reduce
> Issue Type: Bug
> Components: contrib/fair-share
> Affects Versions: 0.20.205.0
> Reporter: Gary Helmling
> Priority: Critical
> Attachments: MAPREDUCE-3206_0.20.205.patch
>
>
> Using the FairScheduler with kerberos authentication does not appear to work with 0.20.205.0. When submitting a job, execution fails when storing the delegation tokens for the job:
> {noformat}
> 11/10/18 21:48:53 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 6 for hdfs on xxxx:8020
> 11/10/18 21:48:53 INFO security.TokenCache: Got dt for hdfs://xxxx/user/hdfs/.staging/job_201110182148_0001;uri=xxxx:8020;t.service=xxxx:8020
> 11/10/18 21:48:53 INFO mapred.FileInputFormat: Total input paths to process : 1
> 11/10/18 21:48:53 INFO mapred.JobClient: Running job: job_201110182148_0001
> 11/10/18 21:48:54 INFO mapred.JobClient: map 0% reduce 0%
> 11/10/18 21:48:54 INFO mapred.JobClient: Job complete: job_201110182148_0001
> 11/10/18 21:48:54 INFO mapred.JobClient: Counters: 0
> 11/10/18 21:48:54 INFO mapred.JobClient: Job Failed: Job initialization failed:
> java.io.IOException: Call to xxxx/xxxx:8020 failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
> at org.apache.hadoop.ipc.Client.wrapException(Client.java:1103)
> at org.apache.hadoop.ipc.Client.call(Client.java:1071)
> at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
> at $Proxy7.getProtocolVersion(Unknown Source)
> at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:396)
> at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:379)
> at org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:118)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:222)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:187)
> at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:89)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1328)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:65)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1346)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:244)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:187)
> at org.apache.hadoop.security.Credentials.writeTokenStorageFile(Credentials.java:175)
> at org.apache.hadoop.mapred.JobInProgress.generateAndStoreTokens(JobInProgress.java:3528)
> at org.apache.hadoop.mapred.JobInProgress.initTasks(JobInProgress.java:696)
> at org.apache.hadoop.mapred.JobTracker.initJob(JobTracker.java:4207)
> at org.apache.hadoop.mapred.FairScheduler$JobInitializer$InitJob.run(FairScheduler.java:291)
> 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: Attempt to obtain new INITIATE credentials failed! (null))]
> at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:539)
> 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:1059)
> at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:484)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:586)
> at org.apache.hadoop.ipc.Client$Connection.access$2000(Client.java:184)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1202)
> at org.apache.hadoop.ipc.Client.call(Client.java:1046)
> ... 21 more
> Caused by: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
> 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:381)
> at org.apache.hadoop.ipc.Client$Connection.access$1100(Client.java:184)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:579)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:576)
> 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:1059)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:575)
> ... 24 more
> Caused by: GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))
> at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:333)
> at sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:128)
> 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)
> ... 33 more
> Caused by: javax.security.auth.login.LoginException: No LoginModules configured for
> at javax.security.auth.login.LoginContext.init(LoginContext.java:256)
> at javax.security.auth.login.LoginContext.<init>(LoginContext.java:499)
> at sun.security.jgss.GSSUtil.login(GSSUtil.java:244)
> at sun.security.jgss.krb5.Krb5Util.getTicket(Krb5Util.java:136)
> at sun.security.jgss.krb5.Krb5InitCredential$1.run(Krb5InitCredential.java:328)
> at java.security.AccessController.doPrivileged(Native Method)
> at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:325)
> ... 40 more
> {noformat}
> The problem seems to have been introduced by the backported changes in MAPREDUCE-2981, which shifted the execution of JobTracker.initJob(), and hence JobInProgress.generateAndStoreTokens(), to underneath the call path for the RPC invocation. As a result, the DFS write in TokenStorage.writeTokenStorageFile() in done under a UGI.doAs() block as the RPC client remote user, without a TGT for negotiating the connection.
> Does this analysis seem right? Previously it seems that JobTracker.initJob() was only called in a separate thread so it was picking up the credentials obtained for the configured JobTracker kerberos principal. The same job runs successfully in a build with MAPREDUCE-2981 reverted.
--
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-3206) FairScheduler breaks writing of
job tokens to MR system dir when using kerberos auth
Posted by "Gary Helmling (Updated) (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/MAPREDUCE-3206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Gary Helmling updated MAPREDUCE-3206:
-------------------------------------
Attachment: MAPREDUCE-3206_0.20.205.patch
This patch executes the token file writing to DFS as the logged in job tracker user. I don't know if there is a better approach, but this resolves the issue for me.
> FairScheduler breaks writing of job tokens to MR system dir when using kerberos auth
> ------------------------------------------------------------------------------------
>
> Key: MAPREDUCE-3206
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-3206
> Project: Hadoop Map/Reduce
> Issue Type: Bug
> Components: contrib/fair-share
> Affects Versions: 0.20.205.0
> Reporter: Gary Helmling
> Priority: Critical
> Attachments: MAPREDUCE-3206_0.20.205.patch
>
>
> Using the FairScheduler with kerberos authentication does not appear to work with 0.20.205.0. When submitting a job, execution fails when storing the delegation tokens for the job:
> {noformat}
> 11/10/18 21:48:53 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 6 for hdfs on xxxx:8020
> 11/10/18 21:48:53 INFO security.TokenCache: Got dt for hdfs://xxxx/user/hdfs/.staging/job_201110182148_0001;uri=xxxx:8020;t.service=xxxx:8020
> 11/10/18 21:48:53 INFO mapred.FileInputFormat: Total input paths to process : 1
> 11/10/18 21:48:53 INFO mapred.JobClient: Running job: job_201110182148_0001
> 11/10/18 21:48:54 INFO mapred.JobClient: map 0% reduce 0%
> 11/10/18 21:48:54 INFO mapred.JobClient: Job complete: job_201110182148_0001
> 11/10/18 21:48:54 INFO mapred.JobClient: Counters: 0
> 11/10/18 21:48:54 INFO mapred.JobClient: Job Failed: Job initialization failed:
> java.io.IOException: Call to xxxx/xxxx:8020 failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
> at org.apache.hadoop.ipc.Client.wrapException(Client.java:1103)
> at org.apache.hadoop.ipc.Client.call(Client.java:1071)
> at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
> at $Proxy7.getProtocolVersion(Unknown Source)
> at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:396)
> at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:379)
> at org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:118)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:222)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:187)
> at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:89)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1328)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:65)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1346)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:244)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:187)
> at org.apache.hadoop.security.Credentials.writeTokenStorageFile(Credentials.java:175)
> at org.apache.hadoop.mapred.JobInProgress.generateAndStoreTokens(JobInProgress.java:3528)
> at org.apache.hadoop.mapred.JobInProgress.initTasks(JobInProgress.java:696)
> at org.apache.hadoop.mapred.JobTracker.initJob(JobTracker.java:4207)
> at org.apache.hadoop.mapred.FairScheduler$JobInitializer$InitJob.run(FairScheduler.java:291)
> 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: Attempt to obtain new INITIATE credentials failed! (null))]
> at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:539)
> 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:1059)
> at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:484)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:586)
> at org.apache.hadoop.ipc.Client$Connection.access$2000(Client.java:184)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1202)
> at org.apache.hadoop.ipc.Client.call(Client.java:1046)
> ... 21 more
> Caused by: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
> 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:381)
> at org.apache.hadoop.ipc.Client$Connection.access$1100(Client.java:184)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:579)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:576)
> 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:1059)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:575)
> ... 24 more
> Caused by: GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))
> at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:333)
> at sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:128)
> 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)
> ... 33 more
> Caused by: javax.security.auth.login.LoginException: No LoginModules configured for
> at javax.security.auth.login.LoginContext.init(LoginContext.java:256)
> at javax.security.auth.login.LoginContext.<init>(LoginContext.java:499)
> at sun.security.jgss.GSSUtil.login(GSSUtil.java:244)
> at sun.security.jgss.krb5.Krb5Util.getTicket(Krb5Util.java:136)
> at sun.security.jgss.krb5.Krb5InitCredential$1.run(Krb5InitCredential.java:328)
> at java.security.AccessController.doPrivileged(Native Method)
> at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:325)
> ... 40 more
> {noformat}
> The problem seems to have been introduced by the backported changes in MAPREDUCE-2981, which shifted the execution of JobTracker.initJob(), and hence JobInProgress.generateAndStoreTokens(), to underneath the call path for the RPC invocation. As a result, the DFS write in TokenStorage.writeTokenStorageFile() in done under a UGI.doAs() block as the RPC client remote user, without a TGT for negotiating the connection.
> Does this analysis seem right? Previously it seems that JobTracker.initJob() was only called in a separate thread so it was picking up the credentials obtained for the configured JobTracker kerberos principal. The same job runs successfully in a build with MAPREDUCE-2981 reverted.
--
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-3206) FairScheduler breaks writing of
job tokens to MR system dir when using kerberos auth
Posted by "Daryn Sharp (Commented) (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/MAPREDUCE-3206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13130679#comment-13130679 ]
Daryn Sharp commented on MAPREDUCE-3206:
----------------------------------------
I'm not familiar with the fair scheduler. However it would seem to me that instead of flipping the user back to the JT, that perhaps the {{UGI.doAs()}} the remote user should be pushed down further in the callstack?
> FairScheduler breaks writing of job tokens to MR system dir when using kerberos auth
> ------------------------------------------------------------------------------------
>
> Key: MAPREDUCE-3206
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-3206
> Project: Hadoop Map/Reduce
> Issue Type: Bug
> Components: contrib/fair-share
> Affects Versions: 0.20.205.0
> Reporter: Gary Helmling
> Priority: Critical
> Attachments: MAPREDUCE-3206_0.20.205.patch
>
>
> Using the FairScheduler with kerberos authentication does not appear to work with 0.20.205.0. When submitting a job, execution fails when storing the delegation tokens for the job:
> {noformat}
> 11/10/18 21:48:53 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 6 for hdfs on xxxx:8020
> 11/10/18 21:48:53 INFO security.TokenCache: Got dt for hdfs://xxxx/user/hdfs/.staging/job_201110182148_0001;uri=xxxx:8020;t.service=xxxx:8020
> 11/10/18 21:48:53 INFO mapred.FileInputFormat: Total input paths to process : 1
> 11/10/18 21:48:53 INFO mapred.JobClient: Running job: job_201110182148_0001
> 11/10/18 21:48:54 INFO mapred.JobClient: map 0% reduce 0%
> 11/10/18 21:48:54 INFO mapred.JobClient: Job complete: job_201110182148_0001
> 11/10/18 21:48:54 INFO mapred.JobClient: Counters: 0
> 11/10/18 21:48:54 INFO mapred.JobClient: Job Failed: Job initialization failed:
> java.io.IOException: Call to xxxx/xxxx:8020 failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
> at org.apache.hadoop.ipc.Client.wrapException(Client.java:1103)
> at org.apache.hadoop.ipc.Client.call(Client.java:1071)
> at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
> at $Proxy7.getProtocolVersion(Unknown Source)
> at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:396)
> at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:379)
> at org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:118)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:222)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:187)
> at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:89)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1328)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:65)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1346)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:244)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:187)
> at org.apache.hadoop.security.Credentials.writeTokenStorageFile(Credentials.java:175)
> at org.apache.hadoop.mapred.JobInProgress.generateAndStoreTokens(JobInProgress.java:3528)
> at org.apache.hadoop.mapred.JobInProgress.initTasks(JobInProgress.java:696)
> at org.apache.hadoop.mapred.JobTracker.initJob(JobTracker.java:4207)
> at org.apache.hadoop.mapred.FairScheduler$JobInitializer$InitJob.run(FairScheduler.java:291)
> 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: Attempt to obtain new INITIATE credentials failed! (null))]
> at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:539)
> 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:1059)
> at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:484)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:586)
> at org.apache.hadoop.ipc.Client$Connection.access$2000(Client.java:184)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1202)
> at org.apache.hadoop.ipc.Client.call(Client.java:1046)
> ... 21 more
> Caused by: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
> 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:381)
> at org.apache.hadoop.ipc.Client$Connection.access$1100(Client.java:184)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:579)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:576)
> 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:1059)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:575)
> ... 24 more
> Caused by: GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))
> at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:333)
> at sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:128)
> 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)
> ... 33 more
> Caused by: javax.security.auth.login.LoginException: No LoginModules configured for
> at javax.security.auth.login.LoginContext.init(LoginContext.java:256)
> at javax.security.auth.login.LoginContext.<init>(LoginContext.java:499)
> at sun.security.jgss.GSSUtil.login(GSSUtil.java:244)
> at sun.security.jgss.krb5.Krb5Util.getTicket(Krb5Util.java:136)
> at sun.security.jgss.krb5.Krb5InitCredential$1.run(Krb5InitCredential.java:328)
> at java.security.AccessController.doPrivileged(Native Method)
> at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:325)
> ... 40 more
> {noformat}
> The problem seems to have been introduced by the backported changes in MAPREDUCE-2981, which shifted the execution of JobTracker.initJob(), and hence JobInProgress.generateAndStoreTokens(), to underneath the call path for the RPC invocation. As a result, the DFS write in TokenStorage.writeTokenStorageFile() in done under a UGI.doAs() block as the RPC client remote user, without a TGT for negotiating the connection.
> Does this analysis seem right? Previously it seems that JobTracker.initJob() was only called in a separate thread so it was picking up the credentials obtained for the configured JobTracker kerberos principal. The same job runs successfully in a build with MAPREDUCE-2981 reverted.
--
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-3206) FairScheduler breaks writing of
job tokens to MR system dir when using kerberos auth
Posted by "Gary Helmling (Commented) (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/MAPREDUCE-3206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13130865#comment-13130865 ]
Gary Helmling commented on MAPREDUCE-3206:
------------------------------------------
I'm not sure pushing down the remote user UGI.doAs() is really possible. As I understand it, the current problem is the UGI.doAs() wrapping the entire RPC call invocation up in org.apache.hadoop.ipc.Server$Handler.run():
{code}
CurCall.set(call);
try {
// Make the call as the user via Subject.doAs, thus associating
// the call with the Subject
if (call.connection.user == null) {
value = call(call.connection.protocol, call.param,
call.timestamp);
} else {
value =
call.connection.user.doAs
(new PrivilegedExceptionAction<Writable>() {
@Override
public Writable run() throws Exception {
// make the call
return call(call.connection.protocol,
call.param, call.timestamp);
}
}
);
}
{code}
Prior to MAPREDUCE-2981, as I understand it, JobTracker.initJob() was being invoked in a separate thread by the scheduler, not under the RPC handler thread. So it was picking up the JobTracker login user since it wasn't running under a UGI.doAs() block.
I haven't checked the FairScheduler code in 0.23 or trunk to see how it's handled there. Maybe that would provide another alternative. Or changing FairScheduler$JobInitializer to run in a separate thread and pull jobs to initialize from a queue might work as well. Maybe that's more along the lines of what you were thinking? I just tried the least intrusive approach.
> FairScheduler breaks writing of job tokens to MR system dir when using kerberos auth
> ------------------------------------------------------------------------------------
>
> Key: MAPREDUCE-3206
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-3206
> Project: Hadoop Map/Reduce
> Issue Type: Bug
> Components: contrib/fair-share
> Affects Versions: 0.20.205.0
> Reporter: Gary Helmling
> Priority: Critical
> Attachments: MAPREDUCE-3206_0.20.205.patch
>
>
> Using the FairScheduler with kerberos authentication does not appear to work with 0.20.205.0. When submitting a job, execution fails when storing the delegation tokens for the job:
> {noformat}
> 11/10/18 21:48:53 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 6 for hdfs on xxxx:8020
> 11/10/18 21:48:53 INFO security.TokenCache: Got dt for hdfs://xxxx/user/hdfs/.staging/job_201110182148_0001;uri=xxxx:8020;t.service=xxxx:8020
> 11/10/18 21:48:53 INFO mapred.FileInputFormat: Total input paths to process : 1
> 11/10/18 21:48:53 INFO mapred.JobClient: Running job: job_201110182148_0001
> 11/10/18 21:48:54 INFO mapred.JobClient: map 0% reduce 0%
> 11/10/18 21:48:54 INFO mapred.JobClient: Job complete: job_201110182148_0001
> 11/10/18 21:48:54 INFO mapred.JobClient: Counters: 0
> 11/10/18 21:48:54 INFO mapred.JobClient: Job Failed: Job initialization failed:
> java.io.IOException: Call to xxxx/xxxx:8020 failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
> at org.apache.hadoop.ipc.Client.wrapException(Client.java:1103)
> at org.apache.hadoop.ipc.Client.call(Client.java:1071)
> at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
> at $Proxy7.getProtocolVersion(Unknown Source)
> at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:396)
> at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:379)
> at org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:118)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:222)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:187)
> at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:89)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1328)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:65)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1346)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:244)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:187)
> at org.apache.hadoop.security.Credentials.writeTokenStorageFile(Credentials.java:175)
> at org.apache.hadoop.mapred.JobInProgress.generateAndStoreTokens(JobInProgress.java:3528)
> at org.apache.hadoop.mapred.JobInProgress.initTasks(JobInProgress.java:696)
> at org.apache.hadoop.mapred.JobTracker.initJob(JobTracker.java:4207)
> at org.apache.hadoop.mapred.FairScheduler$JobInitializer$InitJob.run(FairScheduler.java:291)
> 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: Attempt to obtain new INITIATE credentials failed! (null))]
> at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:539)
> 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:1059)
> at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:484)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:586)
> at org.apache.hadoop.ipc.Client$Connection.access$2000(Client.java:184)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1202)
> at org.apache.hadoop.ipc.Client.call(Client.java:1046)
> ... 21 more
> Caused by: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
> 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:381)
> at org.apache.hadoop.ipc.Client$Connection.access$1100(Client.java:184)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:579)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:576)
> 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:1059)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:575)
> ... 24 more
> Caused by: GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))
> at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:333)
> at sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:128)
> 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)
> ... 33 more
> Caused by: javax.security.auth.login.LoginException: No LoginModules configured for
> at javax.security.auth.login.LoginContext.init(LoginContext.java:256)
> at javax.security.auth.login.LoginContext.<init>(LoginContext.java:499)
> at sun.security.jgss.GSSUtil.login(GSSUtil.java:244)
> at sun.security.jgss.krb5.Krb5Util.getTicket(Krb5Util.java:136)
> at sun.security.jgss.krb5.Krb5InitCredential$1.run(Krb5InitCredential.java:328)
> at java.security.AccessController.doPrivileged(Native Method)
> at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:325)
> ... 40 more
> {noformat}
> The problem seems to have been introduced by the backported changes in MAPREDUCE-2981, which shifted the execution of JobTracker.initJob(), and hence JobInProgress.generateAndStoreTokens(), to underneath the call path for the RPC invocation. As a result, the DFS write in TokenStorage.writeTokenStorageFile() in done under a UGI.doAs() block as the RPC client remote user, without a TGT for negotiating the connection.
> Does this analysis seem right? Previously it seems that JobTracker.initJob() was only called in a separate thread so it was picking up the credentials obtained for the configured JobTracker kerberos principal. The same job runs successfully in a build with MAPREDUCE-2981 reverted.
--
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-3206) FairScheduler breaks writing of
job tokens to MR system dir when using kerberos auth
Posted by "Daryn Sharp (Commented) (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/MAPREDUCE-3206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13130924#comment-13130924 ]
Daryn Sharp commented on MAPREDUCE-3206:
----------------------------------------
That begs the question: was MAPREDUCE-2981 ported incorrectly? Otherwise why doesn't 23/trunk have this issue? If there is just a fundamental difference in 23 vs 205, the minimalist approach is probably best for 205.
> FairScheduler breaks writing of job tokens to MR system dir when using kerberos auth
> ------------------------------------------------------------------------------------
>
> Key: MAPREDUCE-3206
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-3206
> Project: Hadoop Map/Reduce
> Issue Type: Bug
> Components: contrib/fair-share
> Affects Versions: 0.20.205.0
> Reporter: Gary Helmling
> Priority: Critical
> Attachments: MAPREDUCE-3206_0.20.205.patch
>
>
> Using the FairScheduler with kerberos authentication does not appear to work with 0.20.205.0. When submitting a job, execution fails when storing the delegation tokens for the job:
> {noformat}
> 11/10/18 21:48:53 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 6 for hdfs on xxxx:8020
> 11/10/18 21:48:53 INFO security.TokenCache: Got dt for hdfs://xxxx/user/hdfs/.staging/job_201110182148_0001;uri=xxxx:8020;t.service=xxxx:8020
> 11/10/18 21:48:53 INFO mapred.FileInputFormat: Total input paths to process : 1
> 11/10/18 21:48:53 INFO mapred.JobClient: Running job: job_201110182148_0001
> 11/10/18 21:48:54 INFO mapred.JobClient: map 0% reduce 0%
> 11/10/18 21:48:54 INFO mapred.JobClient: Job complete: job_201110182148_0001
> 11/10/18 21:48:54 INFO mapred.JobClient: Counters: 0
> 11/10/18 21:48:54 INFO mapred.JobClient: Job Failed: Job initialization failed:
> java.io.IOException: Call to xxxx/xxxx:8020 failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
> at org.apache.hadoop.ipc.Client.wrapException(Client.java:1103)
> at org.apache.hadoop.ipc.Client.call(Client.java:1071)
> at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
> at $Proxy7.getProtocolVersion(Unknown Source)
> at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:396)
> at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:379)
> at org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:118)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:222)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:187)
> at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:89)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1328)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:65)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1346)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:244)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:187)
> at org.apache.hadoop.security.Credentials.writeTokenStorageFile(Credentials.java:175)
> at org.apache.hadoop.mapred.JobInProgress.generateAndStoreTokens(JobInProgress.java:3528)
> at org.apache.hadoop.mapred.JobInProgress.initTasks(JobInProgress.java:696)
> at org.apache.hadoop.mapred.JobTracker.initJob(JobTracker.java:4207)
> at org.apache.hadoop.mapred.FairScheduler$JobInitializer$InitJob.run(FairScheduler.java:291)
> 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: Attempt to obtain new INITIATE credentials failed! (null))]
> at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:539)
> 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:1059)
> at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:484)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:586)
> at org.apache.hadoop.ipc.Client$Connection.access$2000(Client.java:184)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1202)
> at org.apache.hadoop.ipc.Client.call(Client.java:1046)
> ... 21 more
> Caused by: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))]
> 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:381)
> at org.apache.hadoop.ipc.Client$Connection.access$1100(Client.java:184)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:579)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:576)
> 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:1059)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:575)
> ... 24 more
> Caused by: GSSException: No valid credentials provided (Mechanism level: Attempt to obtain new INITIATE credentials failed! (null))
> at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:333)
> at sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:128)
> 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)
> ... 33 more
> Caused by: javax.security.auth.login.LoginException: No LoginModules configured for
> at javax.security.auth.login.LoginContext.init(LoginContext.java:256)
> at javax.security.auth.login.LoginContext.<init>(LoginContext.java:499)
> at sun.security.jgss.GSSUtil.login(GSSUtil.java:244)
> at sun.security.jgss.krb5.Krb5Util.getTicket(Krb5Util.java:136)
> at sun.security.jgss.krb5.Krb5InitCredential$1.run(Krb5InitCredential.java:328)
> at java.security.AccessController.doPrivileged(Native Method)
> at sun.security.jgss.krb5.Krb5InitCredential.getTgt(Krb5InitCredential.java:325)
> ... 40 more
> {noformat}
> The problem seems to have been introduced by the backported changes in MAPREDUCE-2981, which shifted the execution of JobTracker.initJob(), and hence JobInProgress.generateAndStoreTokens(), to underneath the call path for the RPC invocation. As a result, the DFS write in TokenStorage.writeTokenStorageFile() in done under a UGI.doAs() block as the RPC client remote user, without a TGT for negotiating the connection.
> Does this analysis seem right? Previously it seems that JobTracker.initJob() was only called in a separate thread so it was picking up the credentials obtained for the configured JobTracker kerberos principal. The same job runs successfully in a build with MAPREDUCE-2981 reverted.
--
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