[ https://issues.apache.org/jira/browse/SPARK-26385?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17082896#comment-17082896 ]
Zhou Jiashuai commented on SPARK-26385: --------------------------------------- I enable the log with -Dsun.security.krb5.debug=true and -Dsun.security.spnego.debug=true and get the following logs. It seems to have logged out after run 24 or 25 hours. {quote}[UnixLoginModule]: succeeded importing info: uid = 3107 gid = 3107 supp gid = 3107 Debug is true storeKey false useTicketCache true useKeyTab false doNotPrompt true ticketCache is null isInitiator true KeyTab is null refreshKrb5Config is false principal is null tryFirstPass is false useFirstPass is false storePass is false clearPass is false Acquire TGT from Cache Principal is null null credentials from Ticket Cache [Krb5LoginModule] authentication failed Unable to obtain Principal Name for authentication [UnixLoginModule]: added UnixPrincipal, UnixNumericUserPrincipal, UnixNumericGroupPrincipal(s), to Subject Debug is true storeKey true useTicketCache false useKeyTab true doNotPrompt true ticketCache is null isInitiator true KeyTab is username.keytab-a0d905e9-3926-422f-8068-ffec9ace4cc2 refreshKrb5Config is true principal is usern...@bdp.com tryFirstPass is false useFirstPass is false storePass is false clearPass is false Refreshing Kerberos configuration principal is usern...@bdp.com Will use keytab Commit Succeeded Debug is true storeKey true useTicketCache false useKeyTab true doNotPrompt true ticketCache is null isInitiator true KeyTab is username.keytab-a0d905e9-3926-422f-8068-ffec9ace4cc2 refreshKrb5Config is true principal is usern...@bdp.com tryFirstPass is false useFirstPass is false storePass is false clearPass is false Refreshing Kerberos configuration principal is usern...@bdp.com Will use keytab Commit Succeeded [Krb5LoginModule]: Entering logout [Krb5LoginModule]: logged out Subject Debug is true storeKey true useTicketCache false useKeyTab true doNotPrompt true ticketCache is null isInitiator true KeyTab is username.keytab-a0d905e9-3926-422f-8068-ffec9ace4cc2 refreshKrb5Config is true principal is usern...@bdp.com tryFirstPass is false useFirstPass is false storePass is false clearPass is false Refreshing Kerberos configuration principal is usern...@bdp.com Will use keytab Commit Succeeded {quote} > 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: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org