[
https://issues.apache.org/jira/browse/SPARK-26385?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17088785#comment-17088785
]
Rajeev Kumar commented on SPARK-26385:
--------------------------------------
I am also facing the same issue for spark-2.4.4-bin-hadoop2.7. I am using spark
structured streaming with Kafka. Reading the stream from Kafka and saving it to
HBase.I am also facing the same issue for spark-2.4.4-bin-hadoop2.7. I am using
spark structured streaming with Kafka. Reading the stream from Kafka and saving
it to HBase.
I am putting the logs from my application (after removing ip and username).
When application starts it prints this log. We can see it is creating the
HDFS_DELEGATION_TOKEN (token id = 6972072)
{quote}
20/03/17 13:24:09 INFO DFSClient: Created HDFS_DELEGATION_TOKEN token 6972072
for <user> on ha-hdfs:<name>20/03/17 13:24:09 INFO
HadoopFSDelegationTokenProvider: getting token for:
DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_631280530_1,
ugi=<user>@<abc.com> (auth:KERBEROS)]]20/03/17 13:24:09 INFO DFSClient: Created
HDFS_DELEGATION_TOKEN token 6972073 for <user> on ha-hdfs:<name>20/03/17
13:24:09 INFO HadoopFSDelegationTokenProvider: Renewal interval is 86400039 for
token HDFS_DELEGATION_TOKEN20/03/17 13:24:10 DEBUG
HadoopDelegationTokenManager: Service hive does not require a token. Check your
configuration to see if security is disabled or not.20/03/17 13:24:11 DEBUG
HBaseDelegationTokenProvider: Attempting to fetch HBase security token.20/03/17
13:24:12 INFO HBaseDelegationTokenProvider: Get token from HBase: Kind:
HBASE_AUTH_TOKEN, Service: 812777c29d67, Ident:
(org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier@d1)20/03/17
13:24:12 INFO AMCredentialRenewer: Scheduling login from keytab in 18.0 h.
{quote}
After 18 hours as mentioned in log, it created new tokens also. Token number is
increased (7041621).
{quote}
20/03/18 07:24:10 INFO AMCredentialRenewer: Attempting to login to KDC using
principal: <user>20/03/18 07:24:10 INFO AMCredentialRenewer: Successfully
logged into KDC.20/03/18 07:24:16 DEBUG HadoopFSDelegationTokenProvider:
Delegation token renewer is: rm/<host_name>@<abc.com>20/03/18 07:24:16 INFO
HadoopFSDelegationTokenProvider: getting token for:
DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_-2072296893_22,
ugi=<user>@<abc.com> (auth:KERBEROS)]]20/03/18 07:24:16 INFO DFSClient:
Created HDFS_DELEGATION_TOKEN token 7041621 for <user> on
ha-hdfs:<name>20/03/18 07:24:16 DEBUG HadoopDelegationTokenManager: Service
hive does not require a token. Check your configuration to see if security is
disabled or not.20/03/18 07:24:16 DEBUG HBaseDelegationTokenProvider:
Attempting to fetch HBase security token.20/03/18 07:24:16 INFO
HBaseDelegationTokenProvider: Get token from HBase: Kind: HBASE_AUTH_TOKEN,
Service: 812777c29d67, Ident:
(org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier@102)20/03/18
07:24:16 INFO AMCredentialRenewer: Scheduling login from keytab in 18.0
h.20/03/18 07:24:16 INFO AMCredentialRenewer: Updating delegation
tokens.20/03/18 07:24:17 INFO SparkHadoopUtil: Updating delegation tokens for
current user.20/03/18 07:24:17 DEBUG SparkHadoopUtil: Adding/updating
delegation tokens List(Kind: HBASE_AUTH_TOKEN, Service: 812777c29d67, Ident:
(org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier@102);
org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier@102, Kind:
HDFS_DELEGATION_TOKEN, Service: ha-hdfs:<name>, Ident: (HDFS_DELEGATION_TOKEN
token 7041621 for <user>); HDFS_DELEGATION_TOKEN token 7041621 for <user>;
Renewer: yarn; Issued: 3/18/20 7:24 AM; Max Date: 3/25/20 7:24 AM)20/03/18
07:24:17 INFO SparkHadoopUtil: Updating delegation tokens for current
user.20/03/18 07:24:17 DEBUG SparkHadoopUtil: Adding/updating delegation tokens
List(Kind: HBASE_AUTH_TOKEN, Service: 812777c29d67, Ident:
(org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier@102);
org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier@102, Kind:
HDFS_DELEGATION_TOKEN, Service: ha-hdfs:<name>, Ident: (HDFS_DELEGATION_TOKEN
token 7041621 for <user>); HDFS_DELEGATION_TOKEN token 7041621 for <user>;
Renewer: yarn; Issued: 3/18/20 7:24 AM; Max Date: 3/25/20 7:24 AM)
{quote}
Everything goes fine till 24 hours. After that I see LeaseRenewer exception.
But it is picking the older token number (6972072).This behaviour is same even
if I use "--conf spark.hadoop.fs.hdfs.impl.disable.cache=true"
{quote}
20/03/18 13:24:28 WARN Client: Exception encountered while connecting to the
server :
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
token (HDFS_DELEGATION_TOKEN token 6972072 for <user>) is expired20/03/18
13:24:28 WARN LeaseRenewer: Failed to renew lease for
[DFSClient_NONMAPREDUCE_631280530_1] for 30 seconds. Will retry shortly
...org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
token (HDFS_DELEGATION_TOKEN token 6972072 for <user>) is expired at
org.apache.hadoop.ipc.Client.call(Client.java:1475) 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.$Proxy10.renewLease(Unknown Source) at
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.renewLease(ClientNamenodeProtocolTranslatorPB.java:590)
at sun.reflect.GeneratedMethodAccessor59.invoke(Unknown Source)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:497) 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.renewLease(Unknown Source) at
org.apache.hadoop.hdfs.DFSClient.renewLease(DFSClient.java:892) at
org.apache.hadoop.hdfs.LeaseRenewer.renew(LeaseRenewer.java:423) at
org.apache.hadoop.hdfs.LeaseRenewer.run(LeaseRenewer.java:448) at
org.apache.hadoop.hdfs.LeaseRenewer.access$700(LeaseRenewer.java:71) at
org.apache.hadoop.hdfs.LeaseRenewer$1.run(LeaseRenewer.java:304) at
java.lang.Thread.run(Thread.java:745) 20/03/18 13:24:29 WARN Client: Exception
encountered while connecting to the server :
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
token (HDFS_DELEGATION_TOKEN token 6972072 for <user>) is expired
{quote}
Now, when the application receives new stream, it fails with the above
mentioned error in earlier comments.
{quote}
User class threw exception:
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
token (HDFS_DELEGATION_TOKEN token 6972072 for <user>) can't be found in cache
{quote}
I have noticed that if the application does not receive new stream, it keeps
running (because it's not doing anything) until it gets new stream. I am
guessing it fails when it tries to write to HDFS checkpointing location.
> YARN - Spark Stateful Structured streaming HDFS_DELEGATION_TOKEN not found in
> cache
> -----------------------------------------------------------------------------------
>
> Key: SPARK-26385
> URL: https://issues.apache.org/jira/browse/SPARK-26385
> Project: Spark
> Issue Type: Bug
> Components: Structured Streaming
> Affects Versions: 2.4.0
> Environment: Hadoop 2.6.0, Spark 2.4.0
> Reporter: T M
> Priority: Major
>
>
> Hello,
>
> I have Spark Structured Streaming job which is runnning on YARN(Hadoop 2.6.0,
> Spark 2.4.0). After 25-26 hours, my job stops working with following error:
> {code:java}
> 2018-12-16 22:35:17 ERROR
> org.apache.spark.internal.Logging$class.logError(Logging.scala:91): Query
> TestQuery[id = a61ce197-1d1b-4e82-a7af-60162953488b, runId =
> a56878cf-dfc7-4f6a-ad48-02cf738ccc2f] terminated with error
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
> token (token for REMOVED: HDFS_DELEGATION_TOKEN owner=REMOVED, renewer=yarn,
> realUser=, issueDate=1544903057122, maxDate=1545507857122,
> sequenceNumber=10314, masterKeyId=344) can't be found in cache at
> org.apache.hadoop.ipc.Client.call(Client.java:1470) at
> org.apache.hadoop.ipc.Client.call(Client.java:1401) at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
> at com.sun.proxy.$Proxy9.getFileInfo(Unknown Source) at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:752)
> at sun.reflect.GeneratedMethodAccessor5.invoke(Unknown Source) 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:187)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> at com.sun.proxy.$Proxy10.getFileInfo(Unknown Source) at
> org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1977) at
> org.apache.hadoop.fs.Hdfs.getFileStatus(Hdfs.java:133) at
> org.apache.hadoop.fs.FileContext$14.next(FileContext.java:1120) at
> org.apache.hadoop.fs.FileContext$14.next(FileContext.java:1116) at
> org.apache.hadoop.fs.FSLinkResolver.resolve(FSLinkResolver.java:90) at
> org.apache.hadoop.fs.FileContext.getFileStatus(FileContext.java:1116) at
> org.apache.hadoop.fs.FileContext$Util.exists(FileContext.java:1581) at
> org.apache.spark.sql.execution.streaming.FileContextBasedCheckpointFileManager.exists(CheckpointFileManager.scala:326)
> at
> org.apache.spark.sql.execution.streaming.HDFSMetadataLog.get(HDFSMetadataLog.scala:142)
> at
> org.apache.spark.sql.execution.streaming.HDFSMetadataLog.add(HDFSMetadataLog.scala:110)
> at
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$runBatch$1.apply$mcV$sp(MicroBatchExecution.scala:544)
> at
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$runBatch$1.apply(MicroBatchExecution.scala:542)
> at
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$runBatch$1.apply(MicroBatchExecution.scala:542)
> at
> org.apache.spark.sql.execution.streaming.MicroBatchExecution.withProgressLocked(MicroBatchExecution.scala:554)
> at
> org.apache.spark.sql.execution.streaming.MicroBatchExecution.org$apache$spark$sql$execution$streaming$MicroBatchExecution$$runBatch(MicroBatchExecution.scala:542)
> at
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply$mcV$sp(MicroBatchExecution.scala:198)
> at
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:166)
> at
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:166)
> at
> org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:351)
> at
> org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:58)
> at
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1.apply$mcZ$sp(MicroBatchExecution.scala:166)
> at
> org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:56)
> at
> org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecution.scala:160)
> at
> org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:279)
> at
> org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:189){code}
>
> ^It is important to notice that I tried usual fix for this kind of problems:^
>
> {code:java}
> --conf "spark.hadoop.fs.hdfs.impl.disable.cache=true"
>
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]