You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Saisai Shao (JIRA)" <ji...@apache.org> on 2018/06/08 06:57:00 UTC

[jira] [Comment Edited] (SPARK-24493) Kerberos Ticket Renewal is failing in Hadoop 2.8+ and Hadoop 3

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

Saisai Shao edited comment on SPARK-24493 at 6/8/18 6:56 AM:
-------------------------------------------------------------

Adding more background, the issue is happened when building Spark with Hadoop 2.8+.

In Spark, we use {{TokenIdentifer}} return by {{decodeIdentifier}} to get renew interval for HDFS token, but due to missing service loader file, Hadoop failed to create {{TokenIdentifier}} and returns *null*, which leads to an unexpected renew interval (Long.MaxValue).

The related code in Hadoop is:

{code}
 private static Class<? extends TokenIdentifier>
 getClassForIdentifier(Text kind) { Class<? extends TokenIdentifier> cls = null; synchronized (Token.class) { if (tokenKindMap == null)

{ tokenKindMap = Maps.newHashMap(); for (TokenIdentifier id : ServiceLoader.load(TokenIdentifier.class)) \\{ tokenKindMap.put(id.getKind(), id.getClass()); }

}
 cls = tokenKindMap.get(kind);
 } if (cls == null)

{ LOG.debug("Cannot find class for token kind " + kind); return null; }

return cls;
 }
{code}

The problem is:

The HDFS "DelegationTokenIdentifier" class is in hadoop-hdfs-client jar, but the service loader description file "META-INF/services/org.apache.hadoop.security.token.TokenIdentifier" is in hadoop-hdfs jar. Spark local submit process/driver process (depends on client or cluster mode) only relies on hadoop-hdfs-client jar, but not hadoop-hdfs jar. So the ServiceLoader will be failed to find HDFS "DelegationTokenIdentifier" class and return null.

The issue is due to the change in HADOOP-6200.  Previously we only have building profile for Hadoop 2.6 and 2.7, so there's no issue here. But currently we has a building profile for Hadoop 3.1, so this will fail the token renew in Hadoop 3.1.

The is a Hadoop issue, creating a Spark Jira to track this issue and bump the version when Hadoop side is fixed.


was (Author: jerryshao):
Adding more background, the issue is happened when building Spark with Hadoop 2.8+.

In Spark, we use {{TokenIdentifer}} return by {{decodeIdentifier}} to get renew interval for HDFS token, but due to missing service loader file, Hadoop failed to create {{TokenIdentifier}} and returns *null*, which leads to an unexpected renew interval (Long.MaxValue).

The related code in Hadoop is:
  private static Class<? extends TokenIdentifier>
      getClassForIdentifier(Text kind) {    Class<? extends TokenIdentifier> cls = null;    synchronized (Token.class) {      if (tokenKindMap == null) {
        tokenKindMap = Maps.newHashMap();        for (TokenIdentifier id : ServiceLoader.load(TokenIdentifier.class)) \{
          tokenKindMap.put(id.getKind(), id.getClass());
        }
      }
      cls = tokenKindMap.get(kind);
    }    if (cls == null) {
      LOG.debug("Cannot find class for token kind " + kind);      return null;
    }    return cls;
  }


The problem is:

The HDFS "DelegationTokenIdentifier" class is in hadoop-hdfs-client jar, but the service loader description file "META-INF/services/org.apache.hadoop.security.token.TokenIdentifier" is in hadoop-hdfs jar. Spark local submit process/driver process (depends on client or cluster mode) only relies on hadoop-hdfs-client jar, but not hadoop-hdfs jar. So the ServiceLoader will be failed to find HDFS "DelegationTokenIdentifier" class and return null.

The issue is due to the change in HADOOP-6200.  Previously we only have building profile for Hadoop 2.6 and 2.7, so there's no issue here. But currently we has a building profile for Hadoop 3.1, so this will fail the token renew in Hadoop 3.1.

The is a Hadoop issue, creating a Spark Jira to track this issue and bump the version when Hadoop side is fixed.

> Kerberos Ticket Renewal is failing in Hadoop 2.8+ and Hadoop 3
> --------------------------------------------------------------
>
>                 Key: SPARK-24493
>                 URL: https://issues.apache.org/jira/browse/SPARK-24493
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core, YARN
>    Affects Versions: 2.3.0
>            Reporter: Asif M
>            Priority: Major
>
> Kerberos Ticket Renewal is failing on long running spark job. I have added below 2 kerberos properties in the HDFS configuration and ran a spark streaming job ([hdfs_wordcount.py|https://github.com/apache/spark/blob/master/examples/src/main/python/streaming/hdfs_wordcount.py])
> {noformat}
> dfs.namenode.delegation.token.max-lifetime=1800000 (30min)
> dfs.namenode.delegation.token.renew-interval=900000 (15min)
> {noformat}
>  
> Spark Job failed at 15min with below error:
> {noformat}
> 18/06/04 18:56:51 INFO DAGScheduler: ShuffleMapStage 10896 (call at /usr/hdp/current/spark2-client/python/lib/py4j-0.10.7-src.zip/py4j/java_gateway.py:2381) failed in 0.218 s due to Job aborted due to stage failure: Task 0 in stage 10896.0 failed 4 times, most recent failure: Lost task 0.3 in stage 10896.0 (TID 7290, <GatewayNodeHostname>, executor 1): org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken): token (token for abcd: HDFS_DELEGATION_TOKEN owner=abcd@EXAMPLE.COM, renewer=yarn, realUser=, issueDate=1528136773875, maxDate=1528138573875, sequenceNumber=38, masterKeyId=6) is expired, current time: 2018-06-04 18:56:51,276+0000 expected renewal time: 2018-06-04 18:56:13,875+0000
> at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1499)
> at org.apache.hadoop.ipc.Client.call(Client.java:1445)
> at org.apache.hadoop.ipc.Client.call(Client.java:1355)
> at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:228)
> at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:116)
> at com.sun.proxy.$Proxy18.getBlockLocations(Unknown Source)
> at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(ClientNamenodeProtocolTranslatorPB.java:317)
> 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:422)
> at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:165)
> at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:157)
> at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
> at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:359)
> at com.sun.proxy.$Proxy19.getBlockLocations(Unknown Source)
> at org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:856)
> at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:845)
> at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:834)
> at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:998)
> at org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:326)
> at org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:322)
> at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> at org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:334)
> at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:950)
> at org.apache.hadoop.mapreduce.lib.input.LineRecordReader.initialize(LineRecordReader.java:86)
> at org.apache.spark.rdd.NewHadoopRDD$$anon$1.liftedTree1$1(NewHadoopRDD.scala:189)
> at org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(NewHadoopRDD.scala:186)
> at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:141)
> at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:70)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:105)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:64)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.api.python.PairwiseRDD.compute(PythonRDD.scala:99)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
> at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
> at org.apache.spark.scheduler.Task.run(Task.scala:109)
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> {noformat}
> *Steps to Reproduce:*
>  # Add {{dfs.namenode.delegation.token.max-lifetime}} and {{dfs.namenode.delegation.token.renew-interval}} properties in the HDFS config and restart the affected services.
>  # Run spark streaming job on gateway node of the cluster in one terminal tab
> {noformat}
> /bin/spark-submit --master yarn --principal <SPN> --keytab <Keytab File Full Path> hdfs_wordcount.py "/tmp/streaming_input" 2>&1 | tee driver.log{noformat}
> After 15min the spark application is terminated with the above mentioned error.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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