[
https://issues.apache.org/jira/browse/OOZIE-2871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16212613#comment-16212613
]
Péter Gergő Barna commented on OOZIE-2871:
------------------------------------------
I have the same problem in a similar Env:
Oozie version :4.2.0
Hadoop version:2.7.3
Both Oozie and Hadoop are enabled kerberos.
NN is running in HA in both clusters.
The issue initially was that Oozie distcp action between two secure, HA cluster
failed with the following Exception:
Caused by: GSSException: No valid credentials provided (Mechanism level: Failed
to find any Kerberos tgt)
Cli hadoop distcp is working. I've searched for related Apache issues, this
seems to be the most relevant https://issues.apache.org/jira/browse/OOZIE-2871
After analyzing implementations of other Oozie actions, patches related to
delegation tokens in the upstream, and the cli hadoop distcp implementation I
came up with a patch, which I will summarize briefly
- backport HdfsCredentials, JHSCredentials, YarnRMCredentials,
HadoopTokenHelper and the delegation token creation logic in JavaActionExecutor
from master
- in DistcpActionExecutor make sure that path is extracted from DistCp args and
added to the job conf
- when creating delegation tokens for HDFS, make sure to call
fileSystem.getDelegationToken(renewer); for all different filesystems (coming
from the DistCp paths arguments)
This patch made the original exception go away however an other exception was
raised: Delegation Token can be issued only with kerberos or web authentication
(stacktrace bellow)
>From analyzing the hadoop code in the stacktrace it seemed that this Exception
>is thrown because the UserGroupInformation class is not initialized properly
>and UserGroupInformation.getCurrentUser().getAuthenticationMethod() ==
>"SIMPLE" when it should be "KERBEROS"
Then I have tried to make sure LauncherMapper initializes the UGI properly, so
LauncherMapper is also affected in the patch because of this second exception.
The LauncherMapper modification did not fix the second exception, so I am
still trying to solve this.
I attach this patch, which is a work in progress, any suggestions are welcomed.
{noformat}
WARN org.apache.hadoop.io.retry.RetryInvocationHandler - Exception while
invoking ClientNamenodeProtocolTranslatorPB.getDelegationToken over
bbgnode2.openstacklocal/172.26.120.166:8020. Not retrying because try once and
fail.
org.apache.hadoop.ipc.RemoteException(java.io.IOException): Delegation Token
can be issued only with kerberos or web authentication
at
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getDelegationToken(FSNamesystem.java:6847)
at
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getDelegationToken(NameNodeRpcServer.java:656)
at
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getDelegationToken(ClientNamenodeProtocolServerSideTranslatorPB.java:995)
at
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:982)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2211)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2207)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1724)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2205)
at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1552)
at org.apache.hadoop.ipc.Client.call(Client.java:1496)
at org.apache.hadoop.ipc.Client.call(Client.java:1396)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233)
at com.sun.proxy.$Proxy14.getDelegationToken(Unknown Source)
at
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getDelegationToken(ClientNamenodeProtocolTranslatorPB.java:978)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:278)
at
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:194)
at
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:176)
at com.sun.proxy.$Proxy15.getDelegationToken(Unknown Source)
at org.apache.hadoop.hdfs.DFSClient.getDelegationToken(DFSClient.java:1039)
at
org.apache.hadoop.hdfs.DistributedFileSystem.getDelegationToken(DistributedFileSystem.java:1669)
at org.apache.hadoop.fs.FileSystem.collectDelegationTokens(FileSystem.java:545)
at org.apache.hadoop.fs.FileSystem.addDelegationTokens(FileSystem.java:523)
at
org.apache.hadoop.hdfs.DistributedFileSystem.addDelegationTokens(DistributedFileSystem.java:2381)
at
org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:140)
at
org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:100)
at
org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodes(TokenCache.java:80)
at
org.apache.hadoop.tools.mapred.CopyOutputFormat.checkOutputSpecs(CopyOutputFormat.java:124)
at org.apache.hadoop.mapreduce.JobSubmitter.checkSpecs(JobSubmitter.java:266)
at
org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:139)
at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1290)
at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1287)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1724)
at org.apache.hadoop.mapreduce.Job.submit(Job.java:1287)
at org.apache.hadoop.tools.DistCp.createAndSubmitJob(DistCp.java:199)
at org.apache.hadoop.tools.DistCp.execute(DistCp.java:161)
at org.apache.hadoop.tools.DistCp.run(DistCp.java:134)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
at org.apache.oozie.action.hadoop.DistcpMain.run(DistcpMain.java:78)
at org.apache.oozie.action.hadoop.LauncherMain.run(LauncherMain.java:51)
at org.apache.oozie.action.hadoop.DistcpMain.main(DistcpMain.java:47)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at org.apache.oozie.action.hadoop.LauncherMapper.map(LauncherMapper.java:242)
at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:54)
at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:453)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:343)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:265)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1724)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:259)
{noformat}
> when Enable Kerberos, Oozie perform tasks throw “Client cannot authenticate
> via:[TOKEN, KERBEROS]”
> ---------------------------------------------------------------------------------------------------
>
> Key: OOZIE-2871
> URL: https://issues.apache.org/jira/browse/OOZIE-2871
> Project: Oozie
> Issue Type: Bug
> Components: security
> Affects Versions: 4.2.0
> Environment: Oozie version :4.2.0
> Hadoop version:2.7.2
> Both Oozie and Hadoop are enabled kerberos.
> Reporter: yangfang
> Priority: Critical
> Attachments: secure_multicluster_distcp_workflow.xml
>
>
> When Oozie and Hadoop both enabled kerberos, I submitted a mapreduce job
> to oozie,then I got the error as below:
> 2017-04-27 13:37:12,677 WARN MapReduceActionExecutor: 523 - SERVER[zdh143]
> USER[mr] GROUP[-] TOKEN[] APP[map-reduce-wf]
> JOB[0000008-170427133546167-oozie-mr-W]
> ACTION[0000008-170427133546167-oozie-mr-W@mr-node] Launcher exception: Failed
> on local exception: java.io.IOException:
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate
> via:[TOKEN, KERBEROS]; Host Details : local host is: "zdh142/10.43.183.142";
> destination host is: "zdh143":9000;
> java.io.IOException: Failed on local exception: java.io.IOException:
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate
> via:[TOKEN, KERBEROS]; Host Details : local host is: "zdh142/10.43.183.142";
> destination host is: "zdh143":9000;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1479)
> at org.apache.hadoop.ipc.Client.call(Client.java:1412)
> at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy17.getFileInfo(Unknown Source)
> at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:771)
> at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:606)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:191)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> at com.sun.proxy.$Proxy18.getFileInfo(Unknown Source)
> at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:2108)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1305)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
> at
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1301)
> at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1426)
> at
> org.apache.hadoop.mapred.FileOutputFormat.checkOutputSpecs(FileOutputFormat.java:130)
> at
> org.apache.hadoop.mapreduce.JobSubmitter.checkSpecs(JobSubmitter.java:268)
> at
> org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:139)
> at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1299)
> at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1296)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
> at org.apache.hadoop.mapreduce.Job.submit(Job.java:1296)
> at org.apache.hadoop.mapred.JobClient$1.run(JobClient.java:575)
> at org.apache.hadoop.mapred.JobClient$1.run(JobClient.java:570)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
> at
> org.apache.hadoop.mapred.JobClient.submitJobInternal(JobClient.java:570)
> at org.apache.hadoop.mapred.JobClient.submitJob(JobClient.java:561)
> at
> org.apache.oozie.action.hadoop.MapReduceMain.submitJob(MapReduceMain.java:102)
> at
> org.apache.oozie.action.hadoop.MapReduceMain.run(MapReduceMain.java:64)
> at org.apache.oozie.action.hadoop.LauncherMain.run(LauncherMain.java:47)
> at
> org.apache.oozie.action.hadoop.MapReduceMain.main(MapReduceMain.java:38)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:606)
> at
> org.apache.oozie.action.hadoop.LauncherMapper.map(LauncherMapper.java:238)
> at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:54)
> at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:453)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:343)
> at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:164)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
> at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
> Caused by: java.io.IOException:
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate
> via:[TOKEN, KERBEROS]
> at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:687)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
> at
> org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:650)
> at
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:737)
> at org.apache.hadoop.ipc.Client$Connection.access$2900(Client.java:375)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1528)
> at org.apache.hadoop.ipc.Client.call(Client.java:1451)
> ... 49 more
> Caused by: org.apache.hadoop.security.AccessControlException: Client cannot
> authenticate via:[TOKEN, KERBEROS]
> at
> org.apache.hadoop.security.SaslRpcClient.selectSaslClient(SaslRpcClient.java:172)
> at
> org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:396)
> at
> org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:560)
> at org.apache.hadoop.ipc.Client$Connection.access$1900(Client.java:375)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:729)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:725)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
> at
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:724)
> ... 52 more
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)