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

Marcelo Vanzin commented on SPARK-20328:
----------------------------------------

bq. Since the driver is authenticated, it can request further delegation tokens

No. To create a delegation token you need a TGT. You can't create a delegation 
token just with an existing delegation token. If that were possible, all the 
shenanigans to distribute the user's keytab for long running applications 
wouldn't be needed.

> HadoopRDDs create a MapReduce JobConf, but are not MapReduce jobs
> -----------------------------------------------------------------
>
>                 Key: SPARK-20328
>                 URL: https://issues.apache.org/jira/browse/SPARK-20328
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 2.1.0, 2.1.1, 2.1.2
>            Reporter: Michael Gummelt
>
> In order to obtain {{InputSplit}} information, {{HadoopRDD}} creates a 
> MapReduce {{JobConf}} out of the Hadoop {{Configuration}}: 
> https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala#L138
> Semantically, this is a problem because a HadoopRDD does not represent a 
> Hadoop MapReduce job.  Practically, this is a problem because this line: 
> https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala#L194
>  results in this MapReduce-specific security code being called: 
> https://github.com/apache/hadoop/blob/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java#L130,
>  which assumes the MapReduce master is configured (e.g. via 
> {{yarn.resourcemanager.*}}).  If it isn't, an exception is thrown.
> So I'm seeing this exception thrown as I'm trying to add Kerberos support for 
> the Spark Mesos scheduler:
> {code}
> Exception in thread "main" java.io.IOException: Can't get Master Kerberos 
> principal for use as renewer
>       at 
> org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:116)
>       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.mapred.FileInputFormat.listStatus(FileInputFormat.java:205)
>       at 
> org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:313)
>       at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
> {code}
> I have a workaround where I set a YARN-specific configuration variable to 
> trick {{TokenCache}} into thinking YARN is configured, but this is obviously 
> suboptimal.
> The proper fix to this would likely require significant {{hadoop}} 
> refactoring to make split information available without going through 
> {{JobConf}}, so I'm not yet sure what the best course of action is.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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

Reply via email to