You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Thomas Graves (JIRA)" <ji...@apache.org> on 2017/09/07 16:27:00 UTC

[jira] [Resolved] (SPARK-21890) ObtainCredentials does not pass creds to addDelegationTokens

     [ https://issues.apache.org/jira/browse/SPARK-21890?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Thomas Graves resolved SPARK-21890.
-----------------------------------
       Resolution: Fixed
         Assignee: Sanket Reddy
    Fix Version/s: 2.3.0

> ObtainCredentials does not pass creds to addDelegationTokens
> ------------------------------------------------------------
>
>                 Key: SPARK-21890
>                 URL: https://issues.apache.org/jira/browse/SPARK-21890
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 2.2.0
>            Reporter: Sanket Reddy
>            Assignee: Sanket Reddy
>             Fix For: 2.3.0
>
>
> I observed this while running a oozie job trying to connect to hbase via spark.
> It look like the creds are not being passed in thehttps://github.com/apache/spark/blob/branch-2.2/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala#L53 for 2.2 release.
> More Info as to why it fails on secure grid:
> Oozie client gets the necessary tokens the application needs before launching.  It passes those tokens along to the oozie launcher job (MR job) which will then actually call the Spark client to launch the spark app and pass the tokens along.
> The oozie launcher job cannot get anymore tokens because all it has is tokens ( you can't get tokens with tokens, you need tgt or keytab).  
> The error here is because the launcher job runs the Spark Client to submit the spark job but the spark client doesn't see that it already has the hdfs tokens so it tries to get more, which ends with the exception.
> There was  a change with SPARK-19021 to generalize the hdfs credentials provider that changed it so we don't pass the existing credentials into the call to get tokens so it doesn't realize it already has the necessary tokens.
> Stack trace:
> Warning: Skip remote jar hdfs://axonitered-nn1.red.ygrid.yahoo.com:8020/user/schintap/spark_oozie/apps/lib/spark-starter-2.0-SNAPSHOT-jar-with-dependencies.jar.
> Failing Oozie Launcher, Main class [org.apache.oozie.action.hadoop.SparkMain], main() threw exception, Delegation Token can be issued only with kerberos or web authentication
> 	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getDelegationToken(FSNamesystem.java:5858)
> 	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getDelegationToken(NameNodeRpcServer.java:687)
> 	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getDelegationToken(ClientNamenodeProtocolServerSideTranslatorPB.java:1003)
> 	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:448)
> 	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:999)
> 	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:881)
> 	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:810)
> 	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:1936)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2523)
> 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:5858)
> 	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getDelegationToken(NameNodeRpcServer.java:687)
> 	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getDelegationToken(ClientNamenodeProtocolServerSideTranslatorPB.java:1003)
> 	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:448)
> 	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:999)
> 	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:881)
> 	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:810)
> 	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:1936)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2523)
> 	at org.apache.hadoop.ipc.Client.call(Client.java:1471)
> 	at org.apache.hadoop.ipc.Client.call(Client.java:1408)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:230)
> 	at com.sun.proxy.$Proxy10.getDelegationToken(Unknown Source)
> 	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getDelegationToken(ClientNamenodeProtocolTranslatorPB.java:933)
> 	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:191)
> 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> 	at com.sun.proxy.$Proxy11.getDelegationToken(Unknown Source)
> 	at org.apache.hadoop.hdfs.DFSClient.getDelegationToken(DFSClient.java:1038)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem.getDelegationToken(DistributedFileSystem.java:1543)
> 	at org.apache.hadoop.fs.FileSystem.collectDelegationTokens(FileSystem.java:531)
> 	at org.apache.hadoop.fs.FileSystem.addDelegationTokens(FileSystem.java:509)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem.addDelegationTokens(DistributedFileSystem.java:2228)
> 	at org.apache.spark.deploy.yarn.security.HadoopFSCredentialProvider$$anonfun$obtainCredentials$1.apply(HadoopFSCredentialProvider.scala:53)
> 	at org.apache.spark.deploy.yarn.security.HadoopFSCredentialProvider$$anonfun$obtainCredentials$1.apply(HadoopFSCredentialProvider.scala:50)
> 	at scala.collection.immutable.Set$Set1.foreach(Set.scala:94)
> 	at org.apache.spark.deploy.yarn.security.HadoopFSCredentialProvider.obtainCredentials(HadoopFSCredentialProvider.scala:50)
> 	at org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager$$anonfun$obtainCredentials$2.apply(ConfigurableCredentialManager.scala:82)
> 	at org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager$$anonfun$obtainCredentials$2.apply(ConfigurableCredentialManager.scala:80)
> 	at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
> 	at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
> 	at scala.collection.Iterator$class.foreach(Iterator.scala:893)
> 	at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
> 	at scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
> 	at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
> 	at scala.collection.AbstractTraversable.flatMap(Traversable.scala:104)
> 	at org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager.obtainCredentials(ConfigurableCredentialManager.scala:80)
> 	at org.apache.spark.deploy.yarn.Client.prepareLocalResources(Client.scala:393)
> 	at org.apache.spark.deploy.yarn.Client.createContainerLaunchContext(Client.scala:896)
> 	at org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:173)
> 	at org.apache.spark.deploy.yarn.Client.run(Client.scala:1190)
> 	at org.apache.spark.deploy.yarn.Client$.main(Client.scala:1249)
> 	at org.apache.spark.deploy.yarn.Client.main(Client.scala)
> 	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.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:776)
> 	at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:180)
> 	at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:205)
> 	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:119)
> 	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> 	at org.apache.oozie.action.hadoop.SparkMain.runSpark(SparkMain.java:335)
> 	at org.apache.oozie.action.hadoop.SparkMain.run(SparkMain.java:265)
> 	at org.apache.oozie.action.hadoop.LauncherMain.run(LauncherMain.java:59)
> 	at org.apache.oozie.action.hadoop.SparkMain.main(SparkMain.java:80)
> 	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:235)
> 	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.LocalContainerLauncher$EventHandler.runSubtask(LocalContainerLauncher.java:380)
> 	at org.apache.hadoop.mapred.LocalContainerLauncher$EventHandler.runTask(LocalContainerLauncher.java:301)
> 	at org.apache.hadoop.mapred.LocalContainerLauncher$EventHandler.access$200(LocalContainerLauncher.java:187)
> 	at org.apache.hadoop.mapred.LocalContainerLauncher$EventHandler$1.run(LocalContainerLauncher.java:230)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> I have already done the tests will put up a PR shortly with the updates



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org