You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Kaspar Tint (JIRA)" <ji...@apache.org> on 2018/12/12 10:07:00 UTC

[jira] [Commented] (SPARK-25136) unable to use HDFS checkpoint directories after driver restart

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

Kaspar Tint commented on SPARK-25136:
-------------------------------------

Looks like we bumped into a similar issue with S3 checkpointing. A bunch of queries failed because of usual S3 issues and when restarting one of the failed queries... it ran into a problem fetching the deltafile required to continue processing. 

 

It threw a
{code:java}
IllegalStateException: Error reading delta file s3://some.domain/spark/checkpoints/49/feedback/state/2/11/36870.delta{code}
But when looking into the S3 bucket, it looks like there actually was a delta file but I guess it was not finished yet?

 *{{[s3://some.domain/spark/checkpoints/49/feedback/state/2/11/.36870.delta.02e75d1f-aa48-4c61-9257-a5928b32e22e.TID2469780.tmp|s3://com.twilio.prod.cops-tooling/supernetwork/insights/spark/checkpoints/49/feedback/state/2/11/.36870.delta.02e75d1f-aa48-4c61-9257-a5928b32e22e.TID2469780.tmp]}}*

So it was a dot file with a .tmp suffix and had not been renamed to 36870.delta yet. The query would never be able to recover from this and our application logic would just keep restarting this query until an engineer stepped in and bumped the checkpoint version manually.

 

We can create a safety net for this issue meanwhile and delete the metadata manually if such issue appears...  but it would be nice to have more clarity into what is going on here, why it happened and if it can be fixed on Spark side?

> unable to use HDFS checkpoint directories after driver restart
> --------------------------------------------------------------
>
>                 Key: SPARK-25136
>                 URL: https://issues.apache.org/jira/browse/SPARK-25136
>             Project: Spark
>          Issue Type: Bug
>          Components: Structured Streaming
>    Affects Versions: 2.3.0
>            Reporter: Robert Reid
>            Priority: Major
>
> I have been attempting to work around the issue first discussed at the end of SPARK-20894. The problem that we are encountering is the inability of the Spark Driver to process additional data after restart. Upon restart it reprocesses batches from the initial run. But when processing the first batch it crashes because of missing checkpoint files.
> We have a structured streaming job running on a Spark Standalone cluster. We restart the Spark Driver to renew the Kerberos token in use to access HDFS. Excerpts of log lines from a run are included below. In short, when the new driver starts it reports that it is going to use a new checkpoint directory in HDFS. Workers report that they have loaded a StateStoreProviders that matches the directory. But then the worker reports that it cannot read the delta files. This causes the driver to crash.
> The HDFS directories are present but empty. Further, the directory permissions for the original and new checkpoint directories are the same. The worker never crashes.
> As mentioned in SPARK-20894, deleting the _spark_metadata directory makes subsequent restarts succeed.
> Here is a timeline of log records from a recent run.
> A new run began at 00:29:21. These entries from a worker log look good.
> {{18/08/16 00:30:21 INFO HDFSBackedStateStoreProvider: Retrieved version 0 of HDFSStateStoreProvider[id = (op=0,part=0),dir = hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/a82c35a9-9e98-43f4-9f72-c6e72e4d223e/state/0/0] for update}}
> {{18/08/16 00:30:23 INFO HDFSBackedStateStoreProvider: Committed version 1 for HDFSStateStore[id=(op=0,part=0),dir=hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/a82c35a9-9e98-43f4-9f72-c6e72e4d223e/state/0/0] to file hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/a82c35a9-9e98-43f4-9f72-c6e72e4d223e/state/0/0/1.delta}}
> As the shutdown is occurring the worker reports
> {{18/08/16 00:39:11 INFO HDFSBackedStateStoreProvider: Aborted version 29 for HDFSStateStore[id=(op=0,part=0),dir=hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/a82c35a9-9e98-43f4-9f72-c6e72e4d223e/state/0/0]}}
> The restart began at 00:39:38.
> Driver log entries
> {{18/08/16 00:39:51 INFO MicroBatchExecution: Starting [id = e188d15f-e26a-48fd-9ce6-8c57ce53c2c1, runId = b7ee0163-47db-4392-ab66-94d36ce63074]. Use hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/ead17db8-3660-4bb9-b519-bd7a3599c040 to store the query checkpoint.}}
> {{18/08/16 00:40:26 WARN TaskSetManager: Lost task 0.0 in stage 1.0 (TID 10, 10.251.104.164, executor 3): java.lang.IllegalStateException: Error reading delta file hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/ead17db8-3660-4bb9-b519-bd7a3599c040/state/0/0/1.delta of HDFSStateStoreProvider[id = (op=0,part=0),dir = hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/ead17db8-3660-4bb9-b519-bd7a3599c040/state/0/0]: hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/ead17db8-3660-4bb9-b519-bd7a3599c040/state/0/0/1.delta does not exist}}
> {{Caused by: org.apache.hadoop.ipc.RemoteException(java.io.FileNotFoundException): File does not exist: /projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/ead17db8-3660-4bb9-b519-bd7a3599c040/state/0/0/1.delta}}
>  
> Worker log entries
> {{18/08/16 00:40:26 INFO StateStore: Reported that the loaded instance StateStoreProviderId(StateStoreId(hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/ead17db8-3660-4bb9-b519-bd7a3599c040/state,0,0,default),b7ee0163-47db-4392-ab66-94d36ce63074) is active}}
> {{java.lang.IllegalStateException: Error reading delta file hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/ead17db8-3660-4bb9-b519-bd7a3599c040/state/0/0/1.delta of HDFSStateStoreProvider[id = (op=0,part=0),dir = hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/ead17db8-3660-4bb9-b519-bd7a3599c040/state/0/0]: hdfs://securehdfs/projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/ead17db8-3660-4bb9-b519-bd7a3599c040/state/0/0/1.delta does not exist}}
> {{Caused by: java.io.FileNotFoundException: File does not exist: /projects/flowsnake/search_relevance_proto/search_relevance_proto/63cd3c36-eff9-4900-a010-8eb204429034/checkpoints/ead17db8-3660-4bb9-b519-bd7a3599c040/state/0/0/1.delta}}



--
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