On Thu, Feb 18, 2016 at 6:59 PM, Thomas Lamirault
<thomas.lamira...@ericsson.com> wrote:
> We are trying flink in HA mode.

Great to hear!

> We set in the flink yaml :
>
> state.backend: filesystem
>
> recovery.mode: zookeeper
> recovery.zookeeper.quorum:<Our quorum>
>
> recovery.zookeeper.path.root: <path>
>
> recovery.zookeeper.storageDir: <storageDir>
>
> recovery.backend.fs.checkpointdir: <pathcheckpoint>

It should be state.backend.fs.checkpointdir.

Just to check: Both state.backend.fs.checkpointdir and
recovery.zookeeper.path.root should point to a file system path.

> yarn.application-attempts: 100

This is only relevant if you are using YARN. From your complete


> We want in case of application crash, the pending window has to be restore
> when the application restart.
>
> Pending data are store into the <storageDir>/blob directory ?
>
> Also, we try to write a script who restart the application after exceed the
> max attempts, with the last pending window.
>
> How can I do that ? A simple restart of the application is enough, or do I
> have to "clean" the recovery.zookeeper.path.root ?

Restore happens automatically to the most recently checkpointed state.

Everything under <storageDir> contains the actual state (including
JARs and JobGraph). ZooKeeper contains pointers to this state.
Therefore, you must not delete the ZooKeeper root path.

For the automatic restart, I would recommend using YARN. If you want
to do it manually, you need to restart the JobManager/TaskManager
instances. The application will be recovered automatically from
ZooKeeper/state backend.


Does this help?

– Ufuk

Reply via email to