[
https://issues.apache.org/jira/browse/FLINK-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15379374#comment-15379374
]
ASF GitHub Bot commented on FLINK-4150:
---------------------------------------
GitHub user uce opened a pull request:
https://github.com/apache/flink/pull/2256
[FLINK-4150] [runtime] Don't clean up BlobStore on BlobServer shut down
The `BlobServer` acts as a local cache for uploaded BLOBs. The life-cycle
of each BLOB is bound to the life-cycle of the `BlobServer`. If the BlobServer
shuts down (on JobManager shut down), all local files will be removed.
With HA, BLOBs are persisted to another file system (e.g. HDFS) via the
`BlobStore` in order to have BLOBs available after a JobManager failure (or
shut down). These BLOBs are only allowed to be removed when the job that
requires them enters a globally terminal state (`FINISHED`, `CANCELLED`,
`FAILED`).
This commit removes the `BlobStore` clean up call from the `BlobServer`
shutdown. The `BlobStore` files will only be cleaned up via the
`BlobLibraryCacheManager`'s' clean up task (periodically or on
BlobLibraryCacheManager shutdown). This means that there is a chance that BLOBs
will linger around after the job has terminated, if the job manager fails
before the clean up.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/uce/flink 4150-blobstore
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/2256.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #2256
----
commit 0d4522270881dbbb7164130f47f9d4df617c19c5
Author: Ufuk Celebi <[email protected]>
Date: 2016-07-14T14:29:49Z
[FLINK-4150] [runtime] Don't clean up BlobStore on BlobServer shut down
The `BlobServer` acts as a local cache for uploaded BLOBs. The life-cycle of
each BLOB is bound to the life-cycle of the `BlobServer`. If the BlobServer
shuts down (on JobManager shut down), all local files will be removed.
With HA, BLOBs are persisted to another file system (e.g. HDFS) via the
`BlobStore` in order to have BLOBs available after a JobManager failure (or
shut down). These BLOBs are only allowed to be removed when the job that
requires them enters a globally terminal state (`FINISHED`, `CANCELLED`,
`FAILED`).
This commit removes the `BlobStore` clean up call from the `BlobServer`
shutdown. The `BlobStore` files will only be cleaned up via the
`BlobLibraryCacheManager`'s' clean up task (periodically or on
BlobLibraryCacheManager shutdown). This means that there is a chance that
BLOBs will linger around after the job has terminated, if the job manager
fails before the clean up.
----
> Problem with Blobstore in Yarn HA setting on recovery after cluster shutdown
> ----------------------------------------------------------------------------
>
> Key: FLINK-4150
> URL: https://issues.apache.org/jira/browse/FLINK-4150
> Project: Flink
> Issue Type: Bug
> Components: Job-Submission
> Reporter: Stefan Richter
> Assignee: Ufuk Celebi
> Priority: Blocker
> Fix For: 1.1.0
>
>
> Submitting a job in Yarn with HA can lead to the following exception:
> {code}
> org.apache.flink.streaming.runtime.tasks.StreamTaskException: Cannot load
> user class: org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09
> ClassLoader info: URL ClassLoader:
> file:
> '/tmp/blobStore-ccec0f4a-3e07-455f-945b-4fcd08f5bac1/cache/blob_7fafffe9595cd06aff213b81b5da7b1682e1d6b0'
> (invalid JAR: zip file is empty)
> Class not resolvable through given classloader.
> at
> org.apache.flink.streaming.api.graph.StreamConfig.getStreamOperator(StreamConfig.java:207)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:222)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:588)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> Some job information, including the Blob ids, are stored in Zookeeper. The
> actual Blobs are stored in a dedicated BlobStore, if the recovery mode is set
> to Zookeeper. This BlobStore is typically located in a FS like HDFS. When the
> cluster is shut down, the path for the BlobStore is deleted. When the cluster
> is then restarted, recovering jobs cannot restore because it's Blob ids
> stored in Zookeeper now point to deleted files.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)