[
https://issues.apache.org/jira/browse/FLINK-19154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Till Rohrmann updated FLINK-19154:
----------------------------------
Description:
A user reported that Flink's application mode deletes HA data in case of a
suspended ZooKeeper connection [1].
The problem seems to be that the {{ApplicationDispatcherBootstrap}} class
produces an exception (that the request job can no longer be found because of a
lost ZooKeeper connection) which will be interpreted as a job failure. Due to
this interpretation, the cluster will be shut down with a terminal state of
FAILED which will cause the HA data to be cleaned up. The exact problem occurs
in the {{JobStatusPollingUtils.getJobResult}} which is called by
{{ApplicationDispatcherBootstrap.getJobResult()}}.
The above described behaviour can be found in this log [2].
[1]
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Checkpoint-metadata-deleted-by-Flink-after-ZK-connection-issues-td37937.html
[2] https://pastebin.com/raw/uH9KDU2L
was:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Checkpoint-metadata-deleted-by-Flink-after-ZK-connection-issues-td37937.html
As this mail say , when the application completed with unknown throwable, the
program catch and ignore it , and finally leads to clean up HA data.
``
//catch all throwable,but ignore it.
https://github.com/apache/flink/blob/master/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrap.java#L162
//always use shutDownFuture.complete(status) ,but no
shutDownFuture.completeExceptionally(t)
https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java#L597
// the throwable will always null
https://github.com/apache/flink/blob/6b9cdd41743edd24a929074d62a57b84e7b2dd97/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java#L243
```
So ,if we use shutDownFuture.completeExceptionally(t) when failed because of
an unknown error ,it will not clean up ha data.
I don't know if it was designed to be like this deliberately , so I come here
to find an answer.
By the way, this is the first time I submit an issue , if there are some wrong,
please told me. I am very glad to do something for the community,thanks.
> Always clean up HA data when application completion
> ----------------------------------------------------
>
> Key: FLINK-19154
> URL: https://issues.apache.org/jira/browse/FLINK-19154
> Project: Flink
> Issue Type: Bug
> Components: Client / Job Submission
> Affects Versions: 1.12.0, 1.11.1
> Environment: Run a stand-alone cluster that runs a single job (if you
> are familiar with the way Ververica Platform runs Flink jobs, we use a very
> similar approach). It runs Flink 1.11.1 straight from the official docker
> image.
> Reporter: Husky Zeng
> Priority: Blocker
> Fix For: 1.12.0, 1.11.3
>
>
> A user reported that Flink's application mode deletes HA data in case of a
> suspended ZooKeeper connection [1].
> The problem seems to be that the {{ApplicationDispatcherBootstrap}} class
> produces an exception (that the request job can no longer be found because of
> a lost ZooKeeper connection) which will be interpreted as a job failure. Due
> to this interpretation, the cluster will be shut down with a terminal state
> of FAILED which will cause the HA data to be cleaned up. The exact problem
> occurs in the {{JobStatusPollingUtils.getJobResult}} which is called by
> {{ApplicationDispatcherBootstrap.getJobResult()}}.
> The above described behaviour can be found in this log [2].
> [1]
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Checkpoint-metadata-deleted-by-Flink-after-ZK-connection-issues-td37937.html
> [2] https://pastebin.com/raw/uH9KDU2L
--
This message was sent by Atlassian Jira
(v8.3.4#803005)