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

Jonathan Maron commented on SLIDER-633:
---------------------------------------

Some comments/questions:

1)  Can you double check whether this patch was built against the latest 
develop branch - line numbers don't appear to line up.
2)  Rather than "mapreduce.job.credentials.binary" perhaps it should have a 
slider associated name like "slider.client.credentials.binary"?  Or is that a 
name being leveraged somewhere down stream in a mapreduce job (I don't actually 
see the binary being leveraged as a credential source by slider in this code)?
3)  You may want to re-code to allow for the specification of the property in 
appConfig as well as slider-client.xml.  Having it in slider-client.xml means 
that the same credential binary is leveraged by all applications launched by 
slider, which may be limiting or inappropriate in some instances.
4) I don't quite understand the code in AppMasterLauncher:  if a binary token 
file is specified, we don't add the HDFS delegation tokens to the credentials 
for the AM launch?  How would the AM subsequently access HDFS resources?  There 
are a number of HDFS accesses required prior to actual keytab login on the AM, 
so w/o any HDFS tokens those access attempt would fail, I'd think?


> Slider should support invocation via Oozie
> ------------------------------------------
>
>                 Key: SLIDER-633
>                 URL: https://issues.apache.org/jira/browse/SLIDER-633
>             Project: Slider
>          Issue Type: Improvement
>    Affects Versions: Slider 0.50
>            Reporter: Lee Yang
>         Attachments: fix_oozie_launch.patch
>
>
> In a secure Hadoop installation, when attempting to launch a slider 
> application via an Oozie shell-action, I see the following exception:
> {noformat}
> Stdoutput org.apache.hadoop.ipc.RemoteException(java.io.IOException): 
> Delegation Token can be issued only with kerberos or web authentication
> Stdoutput     at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getDelegationToken(FSNamesystem.java:6757)
> Stdoutput     at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getDelegationToken(NameNodeRpcServer.java:499)
> Stdoutput     at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getDelegationToken(ClientNamenodeProtocolServerSideTranslatorPB.java:921)
> Stdoutput     at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> Stdoutput     at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:585)
> Stdoutput     at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:928)
> Stdoutput     at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2013)
> Stdoutput     at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2009)
> Stdoutput     at java.security.AccessController.doPrivileged(Native Method)
> Stdoutput     at javax.security.auth.Subject.doAs(Subject.java:415)
> Stdoutput     at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1637)
> Stdoutput     at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2007)
> Stdoutput 
> Stdoutput     at org.apache.hadoop.ipc.Client.call(Client.java:1411)
> Stdoutput     at org.apache.hadoop.ipc.Client.call(Client.java:1364)
> Stdoutput     at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
> Stdoutput     at com.sun.proxy.$Proxy17.getDelegationToken(Unknown Source)
> Stdoutput     at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getDelegationToken(ClientNamenodeProtocolTranslatorPB.java:864)
> Stdoutput     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> Stdoutput     at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> Stdoutput     at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> Stdoutput     at java.lang.reflect.Method.invoke(Method.java:601)
> Stdoutput     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
> Stdoutput     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> Stdoutput     at com.sun.proxy.$Proxy18.getDelegationToken(Unknown Source)
> Stdoutput     at 
> org.apache.hadoop.hdfs.DFSClient.getDelegationToken(DFSClient.java:947)
> Stdoutput     at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getDelegationToken(DistributedFileSystem.java:1305)
> Stdoutput     at 
> org.apache.hadoop.fs.FileSystem.collectDelegationTokens(FileSystem.java:527)
> Stdoutput     at 
> org.apache.hadoop.fs.FileSystem.addDelegationTokens(FileSystem.java:505)
> Stdoutput     at 
> org.apache.slider.core.launch.AppMasterLauncher.addSecurityTokens(AppMasterLauncher.java:209)
> Stdoutput     at 
> org.apache.slider.core.launch.AppMasterLauncher.completeAppMasterLaunch(AppMasterLauncher.java:183)
> Stdoutput     at 
> org.apache.slider.core.launch.AppMasterLauncher.submitApplication(AppMasterLauncher.java:214)
> Stdoutput     at 
> org.apache.slider.client.SliderClient.launchApplication(SliderClient.java:1127)
> Stdoutput     at 
> org.apache.slider.client.SliderClient.startCluster(SliderClient.java:771)
> Stdoutput     at 
> org.apache.slider.client.SliderClient.actionCreate(SliderClient.java:515)
> Stdoutput     at 
> org.apache.slider.client.SliderClient.runService(SliderClient.java:295)
> Stdoutput     at 
> org.apache.slider.core.main.ServiceLauncher.launchService(ServiceLauncher.java:186)
> Stdoutput     at 
> org.apache.slider.core.main.ServiceLauncher.launchServiceRobustly(ServiceLauncher.java:471)
> Stdoutput     at 
> org.apache.slider.core.main.ServiceLauncher.launchServiceAndExit(ServiceLauncher.java:401)
> Stdoutput     at 
> org.apache.slider.core.main.ServiceLauncher.serviceMain(ServiceLauncher.java:626)
> Stdoutput     at org.apache.slider.Slider.main(Slider.java:49)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to