You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Encho Mishinev <en...@gmail.com> on 2018/08/27 14:13:12 UTC

JobGraphs not cleaned up in HA mode

I am running Flink 1.5.3 with two job managers and two task managers in Kubernetes along with HDFS and Zookeeper in high-availability mode.

My problem occurs after the following actions:
- Upload a .jar file to jobmanager-1
- Run a streaming job from the jar on jobmanager-1
- Wait for 1 or 2 checkpoints to succeed
- Kill pod of jobmanager-1
After a short delay, jobmanager-2 takes leadership and correctly restores the job and continues it
- Stop job from jobmanager-2

At this point all seems well, but the problem is that jobmanager-2 does not clean up anything that was left from jobmanager-1. This means that both in HDFS and in Zookeeper remain job graphs, which later on obstruct any work of both managers as after any reset they unsuccessfully try to restore a non-existent job and fail over and over again.

I am quite certain that jobmanager-2 does not know about any of jobmanager-1’s files since the Zookeeper logs reveal that it tries to duplicate job folders:

2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0 cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException when processing sessionid:0x1657aa15e480033 type:create cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77 Error:KeeperErrorCode = NodeExists for /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77

2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0 cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException when processing sessionid:0x1657aa15e480033 type:create cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15 Error:KeeperErrorCode = NodeExists for /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15

Also jobmanager-2 attempts to delete the jobgraphs folder in Zookeeper when the job is stopped, but fails since there are leftover files in it from jobmanager-1:

2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0 cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException when processing sessionid:0x1657aa15e480033 type:delete cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15 Error:KeeperErrorCode = Directory not empty for /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15

I’ve noticed that when restoring the job, it seems like jobmanager-2 does not get anything more than jobID, while it perhaps needs some metadata? Here is the log that seems suspicious to me:

2018-08-27 13:09:18,113 INFO  org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).

All other logs seem fine in jobmanager-2, it doesn’t seem to be aware that it’s overwriting anything or not deleting properly.

My question is - what is the intended way for the job managers to correctly exchange metadata in HA mode and why is it not working for me?

Thanks in advance!

回复: JobGraphs not cleaned up in HA mode

Posted by 曾祥才 <xc...@qq.com>.
the chk-* directory is not found , I think the misssing because of jobmanager removes it automaticly , but why it still in zookeeper?&nbsp;&nbsp;


&nbsp;




------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 晚上8:31
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"vino yang"<yanghua1127@gmail.com&gt;;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



One more thing:You configured:
high-availability.cluster-id: /cluster-test

it should be:
high-availability.cluster-id: cluster-test


I don't think this is major issue, in case it helps, you can check.

Can you check one more thing:
Is check pointing happening or not?&nbsp;
Were you able to see the chk-* folder under checkpoint directory?


Regards
Bhaskar


On Thu, Nov 28, 2019 at 5:00 PM 曾祥才 <xcz200706@qq.com&gt; wrote:

hi,
Is there any deference (for me using nas is more convenient to test currently)?&nbsp; &nbsp;
from the docs seems hdfs ,s3, nfs etc all will be fine.






------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"vino yang"<yanghua1127@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 晚上7:17
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Hi,

Why do you not use HDFS directly?


Best,
Vino


曾祥才 <xcz200706@qq.com&gt; 于2019年11月28日周四 下午6:48写道:



anyone have the same problem? pls help, thks






------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:46
收件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;回复: JobGraphs not cleaned up in HA mode



the config&nbsp; (/flink is the NASdirectory ):&nbsp; 


jobmanager.rpc.address: flink-jobmanager
taskmanager.numberOfTaskSlots: 16
web.upload.dir: /flink/webUpload
blob.server.port: 6124
jobmanager.rpc.port: 6123
taskmanager.rpc.port: 6122
jobmanager.heap.size: 1024m
taskmanager.heap.size: 1024m
high-availability: zookeeper
high-availability.cluster-id: /cluster-test
high-availability.storageDir: /flink/ha
high-availability.zookeeper.quorum: ****:2181
high-availability.jobmanager.port: 6123
high-availability.zookeeper.path.root: /flink/risk-insight
high-availability.zookeeper.path.checkpoints: /flink/zk-checkpoints
state.backend: filesystem
state.checkpoints.dir: file:///flink/checkpoints
state.savepoints.dir: file:///flink/savepoints
state.checkpoints.num-retained: 2
jobmanager.execution.failover-strategy: region
jobmanager.archive.fs.dir: file:///flink/archive/history

&nbsp;




------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午3:12
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Can you share the flink configuration once?

Regards
Bhaskar


On Thu, Nov 28, 2019 at 12:09 PM 曾祥才 <xcz200706@qq.com&gt; wrote:

if i clean the zookeeper data , it runs fine .&nbsp; but next time when the jobmanager failed and redeploy the error occurs again








------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午3:05
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;

主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Again it could not find the state store file: "Caused by: java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 "&nbsp;Check why its unable to find. 
Better thing is: Clean up zookeeper state and check your configurations, correct them and restart cluster.
Otherwise it always picks up corrupted state from zookeeper and it will never restart


Regards
Bhaskar


On Thu, Nov 28, 2019 at 11:51 AM 曾祥才 <xcz200706@qq.com&gt; wrote:

i've made a misstake( the log before is another cluster) . the full exception log is :


INFO&nbsp; org.apache.flink.runtime.dispatcher.StandaloneDispatcher&nbsp; &nbsp; &nbsp; - Recovering all persisted jobs. 
2019-11-28 02:33:12,726 INFO&nbsp; org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl&nbsp; - Starting the SlotManager. 
2019-11-28 02:33:12,743 INFO&nbsp; org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore&nbsp; - Released locks of job graph 639170a9d710bacfd113ca66b2aacefa from ZooKeeper. 
2019-11-28 02:33:12,744 ERROR org.apache.flink.runtime.entrypoint.ClusterEntrypoint&nbsp; &nbsp; &nbsp; &nbsp; &nbsp;- Fatal error occurred in the cluster entrypoint. 
org.apache.flink.runtime.dispatcher.DispatcherException: Failed to take leadership with session id 607e1fe0-f1b4-4af0-af16-4137d779defb. 
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$30(Dispatcher.java:915) 
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774) 
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750) 
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) 
	at java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575) 
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:594) 
	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456) 
	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40) 
	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44) 
	at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) 
	at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) 
	at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) 
	at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) 
Caused by: java.lang.RuntimeException: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state handle is broken. Try cleaning the state handle store. 
	at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199) 
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:75) 
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616) 
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591) 
	... 7 more 
Caused by: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state handle is broken. Try cleaning the state handle store. 
	at org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:190) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:755) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:740) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:721) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$27(Dispatcher.java:888) 
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:73) 
	... 9 more 
Caused by: java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 (No such file or directory) 
	at java.io.FileInputStream.open0(Native Method) 
	at java.io.FileInputStream.open(FileInputStream.java:195) 



&nbsp;




------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:46
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Is it filesystem or hadoop? If its NAS then why the exception "Caused by: org.apache.hadoop.hdfs.BlockMissingException: "It seems you configured hadoop state store and giving NAS mount. 


Regards
Bhaskar


&nbsp;


On Thu, Nov 28, 2019 at 11:36 AM 曾祥才 <xcz200706@qq.com&gt; wrote:

/flink/checkpoints&nbsp; is a external persistent store (a nas directory mounts to the job manager)








------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:29
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"user"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Following are the mandatory condition to run in HA:

a) You should have persistent common external store for jobmanager and task managers to while writing the state
b) You should have persistent external store for zookeeper to store the Jobgraph.


Zookeeper is referring&nbsp; path: /flink/checkpoints/submittedJobGraph480ddf9572ed&nbsp; to get the job graph but jobmanager unable to find it.
It seems /flink/checkpoints&nbsp; is not the external persistent store




Regards
Bhaskar


On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xcz200706@qq.com&gt; wrote:

hi ,I've the same problem with flink 1.9.1 , any solution to fix it
 when the k8s redoploy jobmanager ,&nbsp; the error looks like (seems zk not
 remove submitted job info, but jobmanager remove the file):&nbsp; 
 
 
 Caused by: org.apache.flink.util.FlinkException: Could not retrieve
 submitted JobGraph from state handle under
 /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
 handle is broken. Try cleaning the state handle store.
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
 &nbsp; &nbsp; &nbsp; &nbsp; ... 9 more
 Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain
 block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
 
 
 
 --
 Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

Re: JobGraphs not cleaned up in HA mode

Posted by Vijay Bhaskar <bh...@gmail.com>.
One more thing:
You configured:
high-availability.cluster-id: /cluster-test
it should be:
high-availability.cluster-id: cluster-test
I don't think this is major issue, in case it helps, you can check.
Can you check one more thing:
Is check pointing happening or not?
Were you able to see the chk-* folder under checkpoint directory?

Regards
Bhaskar

On Thu, Nov 28, 2019 at 5:00 PM 曾祥才 <xc...@qq.com> wrote:

> hi,
> Is there any deference (for me using nas is more convenient to test
> currently)?
> from the docs seems hdfs ,s3, nfs etc all will be fine.
>
>
>
> ------------------ 原始邮件 ------------------
> *发件人:* "vino yang"<ya...@gmail.com>;
> *发送时间:* 2019年11月28日(星期四) 晚上7:17
> *收件人:* "曾祥才"<xc...@qq.com>;
> *抄送:* "Vijay Bhaskar"<bh...@gmail.com>;"User-Flink"<
> user@flink.apache.org>;
> *主题:* Re: JobGraphs not cleaned up in HA mode
>
> Hi,
>
> Why do you not use HDFS directly?
>
> Best,
> Vino
>
> 曾祥才 <xc...@qq.com> 于2019年11月28日周四 下午6:48写道:
>
>>
>> anyone have the same problem? pls help, thks
>>
>>
>>
>> ------------------ 原始邮件 ------------------
>> *发件人:* "曾祥才"<xc...@qq.com>;
>> *发送时间:* 2019年11月28日(星期四) 下午2:46
>> *收件人:* "Vijay Bhaskar"<bh...@gmail.com>;
>> *抄送:* "User-Flink"<us...@flink.apache.org>;
>> *主题:* 回复: JobGraphs not cleaned up in HA mode
>>
>> the config  (/flink is the NASdirectory ):
>>
>> jobmanager.rpc.address: flink-jobmanager
>> taskmanager.numberOfTaskSlots: 16
>> web.upload.dir: /flink/webUpload
>> blob.server.port: 6124
>> jobmanager.rpc.port: 6123
>> taskmanager.rpc.port: 6122
>> jobmanager.heap.size: 1024m
>> taskmanager.heap.size: 1024m
>> high-availability: zookeeper
>> high-availability.cluster-id: /cluster-test
>> high-availability.storageDir: /flink/ha
>> high-availability.zookeeper.quorum: ****:2181
>> high-availability.jobmanager.port: 6123
>> high-availability.zookeeper.path.root: /flink/risk-insight
>> high-availability.zookeeper.path.checkpoints: /flink/zk-checkpoints
>> state.backend: filesystem
>> state.checkpoints.dir: file:///flink/checkpoints
>> state.savepoints.dir: file:///flink/savepoints
>> state.checkpoints.num-retained: 2
>> jobmanager.execution.failover-strategy: region
>> jobmanager.archive.fs.dir: file:///flink/archive/history
>>
>>
>>
>> ------------------ 原始邮件 ------------------
>> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
>> *发送时间:* 2019年11月28日(星期四) 下午3:12
>> *收件人:* "曾祥才"<xc...@qq.com>;
>> *抄送:* "User-Flink"<us...@flink.apache.org>;
>> *主题:* Re: JobGraphs not cleaned up in HA mode
>>
>> Can you share the flink configuration once?
>>
>> Regards
>> Bhaskar
>>
>> On Thu, Nov 28, 2019 at 12:09 PM 曾祥才 <xc...@qq.com> wrote:
>>
>>> if i clean the zookeeper data , it runs fine .  but next time when the
>>> jobmanager failed and redeploy the error occurs again
>>>
>>>
>>>
>>>
>>> ------------------ 原始邮件 ------------------
>>> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
>>> *发送时间:* 2019年11月28日(星期四) 下午3:05
>>> *收件人:* "曾祥才"<xc...@qq.com>;
>>> *主题:* Re: JobGraphs not cleaned up in HA mode
>>>
>>> Again it could not find the state store file: "Caused by:
>>> java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 "
>>>  Check why its unable to find.
>>> Better thing is: Clean up zookeeper state and check your configurations,
>>> correct them and restart cluster.
>>> Otherwise it always picks up corrupted state from zookeeper and it will
>>> never restart
>>>
>>> Regards
>>> Bhaskar
>>>
>>> On Thu, Nov 28, 2019 at 11:51 AM 曾祥才 <xc...@qq.com> wrote:
>>>
>>>> i've made a misstake( the log before is another cluster) . the full
>>>> exception log is :
>>>>
>>>>
>>>> INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher      -
>>>> Recovering all persisted jobs.
>>>> 2019-11-28 02:33:12,726 INFO
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl  -
>>>> Starting the SlotManager.
>>>> 2019-11-28 02:33:12,743 INFO
>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>> Released locks of job graph 639170a9d710bacfd113ca66b2aacefa from
>>>> ZooKeeper.
>>>> 2019-11-28 02:33:12,744 ERROR
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Fatal error
>>>> occurred in the cluster entrypoint.
>>>> org.apache.flink.runtime.dispatcher.DispatcherException: Failed to take
>>>> leadership with session id 607e1fe0-f1b4-4af0-af16-4137d779defb.
>>>> at
>>>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$30(Dispatcher.java:915)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575)
>>>> at
>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:594)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456)
>>>>
>>>> at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
>>>> at
>>>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44)
>>>>
>>>> at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>>>> at
>>>> akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>>>>
>>>> at
>>>> akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>>>> at
>>>> akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
>>>>
>>>> Caused by: java.lang.RuntimeException:
>>>> org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph
>>>> from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates
>>>> that the retrieved state handle is broken. Try cleaning the state handle
>>>> store.
>>>> at
>>>> org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
>>>> at
>>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:75)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)
>>>> at
>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)
>>>>
>>>> ... 7 more
>>>> Caused by: org.apache.flink.util.FlinkException: Could not retrieve
>>>> submitted JobGraph from state handle under
>>>> /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state
>>>> handle is broken. Try cleaning the state handle store.
>>>> at
>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:190)
>>>>
>>>> at
>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:755)
>>>>
>>>> at
>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:740)
>>>>
>>>> at
>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:721)
>>>>
>>>> at
>>>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$27(Dispatcher.java:888)
>>>>
>>>> at
>>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:73)
>>>>
>>>> ... 9 more
>>>> Caused by: java.io.FileNotFoundException:
>>>> /flink/ha/submittedJobGraph0c6bcff01199 (No such file or directory)
>>>> at java.io.FileInputStream.open0(Native Method)
>>>> at java.io.FileInputStream.open(FileInputStream.java:195)
>>>>
>>>>
>>>>
>>>>
>>>> ------------------ 原始邮件 ------------------
>>>> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
>>>> *发送时间:* 2019年11月28日(星期四) 下午2:46
>>>> *收件人:* "曾祥才"<xc...@qq.com>;
>>>> *抄送:* "User-Flink"<us...@flink.apache.org>;
>>>> *主题:* Re: JobGraphs not cleaned up in HA mode
>>>>
>>>> Is it filesystem or hadoop? If its NAS then why the exception "Caused
>>>> by: org.apache.hadoop.hdfs.BlockMissingException: "
>>>> It seems you configured hadoop state store and giving NAS mount.
>>>>
>>>> Regards
>>>> Bhaskar
>>>>
>>>>
>>>>
>>>> On Thu, Nov 28, 2019 at 11:36 AM 曾祥才 <xc...@qq.com> wrote:
>>>>
>>>>> /flink/checkpoints  is a external persistent store (a nas directory
>>>>> mounts to the job manager)
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> ------------------ 原始邮件 ------------------
>>>>> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
>>>>> *发送时间:* 2019年11月28日(星期四) 下午2:29
>>>>> *收件人:* "曾祥才"<xc...@qq.com>;
>>>>> *抄送:* "user"<us...@flink.apache.org>;
>>>>> *主题:* Re: JobGraphs not cleaned up in HA mode
>>>>>
>>>>> Following are the mandatory condition to run in HA:
>>>>>
>>>>> a) You should have persistent common external store for jobmanager and
>>>>> task managers to while writing the state
>>>>> b) You should have persistent external store for zookeeper to store
>>>>> the Jobgraph.
>>>>>
>>>>> Zookeeper is referring  path:
>>>>> /flink/checkpoints/submittedJobGraph480ddf9572ed  to get the job graph but
>>>>> jobmanager unable to find it.
>>>>> It seems /flink/checkpoints  is not the external persistent store
>>>>>
>>>>>
>>>>> Regards
>>>>> Bhaskar
>>>>>
>>>>> On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xc...@qq.com> wrote:
>>>>>
>>>>>> hi ,I've the same problem with flink 1.9.1 , any solution to fix it
>>>>>> when the k8s redoploy jobmanager ,  the error looks like (seems zk not
>>>>>> remove submitted job info, but jobmanager remove the file):
>>>>>>
>>>>>>
>>>>>> Caused by: org.apache.flink.util.FlinkException: Could not retrieve
>>>>>> submitted JobGraph from state handle under
>>>>>> /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved
>>>>>> state
>>>>>> handle is broken. Try cleaning the state handle store.
>>>>>>         at
>>>>>>
>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
>>>>>>         at
>>>>>>
>>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
>>>>>>         at
>>>>>>
>>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
>>>>>>         at
>>>>>>
>>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
>>>>>>         at
>>>>>>
>>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
>>>>>>         at
>>>>>>
>>>>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
>>>>>>         ... 9 more
>>>>>> Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not
>>>>>> obtain
>>>>>> block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
>>>>>> file=/flink/checkpoints/submittedJobGraph480ddf9572ed
>>>>>>         at
>>>>>>
>>>>>> org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> Sent from:
>>>>>> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
>>>>>>
>>>>>

回复: JobGraphs not cleaned up in HA mode

Posted by 曾祥才 <xc...@qq.com>.
hi,
Is there any deference (for me using nas is more convenient to test currently)?&nbsp; &nbsp;
from the docs seems hdfs ,s3, nfs etc all will be fine.






------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"vino yang"<yanghua1127@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 晚上7:17
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Hi,

Why do you not use HDFS directly?


Best,
Vino


曾祥才 <xcz200706@qq.com&gt; 于2019年11月28日周四 下午6:48写道:



anyone have the same problem? pls help, thks






------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:46
收件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;回复: JobGraphs not cleaned up in HA mode



the config&nbsp; (/flink is the NASdirectory ):&nbsp; 


jobmanager.rpc.address: flink-jobmanager
taskmanager.numberOfTaskSlots: 16
web.upload.dir: /flink/webUpload
blob.server.port: 6124
jobmanager.rpc.port: 6123
taskmanager.rpc.port: 6122
jobmanager.heap.size: 1024m
taskmanager.heap.size: 1024m
high-availability: zookeeper
high-availability.cluster-id: /cluster-test
high-availability.storageDir: /flink/ha
high-availability.zookeeper.quorum: ****:2181
high-availability.jobmanager.port: 6123
high-availability.zookeeper.path.root: /flink/risk-insight
high-availability.zookeeper.path.checkpoints: /flink/zk-checkpoints
state.backend: filesystem
state.checkpoints.dir: file:///flink/checkpoints
state.savepoints.dir: file:///flink/savepoints
state.checkpoints.num-retained: 2
jobmanager.execution.failover-strategy: region
jobmanager.archive.fs.dir: file:///flink/archive/history

&nbsp;




------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午3:12
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Can you share the flink configuration once?

Regards
Bhaskar


On Thu, Nov 28, 2019 at 12:09 PM 曾祥才 <xcz200706@qq.com&gt; wrote:

if i clean the zookeeper data , it runs fine .&nbsp; but next time when the jobmanager failed and redeploy the error occurs again








------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午3:05
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;

主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Again it could not find the state store file: "Caused by: java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 "&nbsp;Check why its unable to find. 
Better thing is: Clean up zookeeper state and check your configurations, correct them and restart cluster.
Otherwise it always picks up corrupted state from zookeeper and it will never restart


Regards
Bhaskar


On Thu, Nov 28, 2019 at 11:51 AM 曾祥才 <xcz200706@qq.com&gt; wrote:

i've made a misstake( the log before is another cluster) . the full exception log is :


INFO&nbsp; org.apache.flink.runtime.dispatcher.StandaloneDispatcher&nbsp; &nbsp; &nbsp; - Recovering all persisted jobs. 
2019-11-28 02:33:12,726 INFO&nbsp; org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl&nbsp; - Starting the SlotManager. 
2019-11-28 02:33:12,743 INFO&nbsp; org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore&nbsp; - Released locks of job graph 639170a9d710bacfd113ca66b2aacefa from ZooKeeper. 
2019-11-28 02:33:12,744 ERROR org.apache.flink.runtime.entrypoint.ClusterEntrypoint&nbsp; &nbsp; &nbsp; &nbsp; &nbsp;- Fatal error occurred in the cluster entrypoint. 
org.apache.flink.runtime.dispatcher.DispatcherException: Failed to take leadership with session id 607e1fe0-f1b4-4af0-af16-4137d779defb. 
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$30(Dispatcher.java:915) 
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774) 
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750) 
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) 
	at java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575) 
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:594) 
	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456) 
	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40) 
	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44) 
	at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) 
	at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) 
	at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) 
	at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) 
Caused by: java.lang.RuntimeException: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state handle is broken. Try cleaning the state handle store. 
	at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199) 
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:75) 
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616) 
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591) 
	... 7 more 
Caused by: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state handle is broken. Try cleaning the state handle store. 
	at org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:190) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:755) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:740) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:721) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$27(Dispatcher.java:888) 
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:73) 
	... 9 more 
Caused by: java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 (No such file or directory) 
	at java.io.FileInputStream.open0(Native Method) 
	at java.io.FileInputStream.open(FileInputStream.java:195) 



&nbsp;




------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:46
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Is it filesystem or hadoop? If its NAS then why the exception "Caused by: org.apache.hadoop.hdfs.BlockMissingException: "It seems you configured hadoop state store and giving NAS mount. 


Regards
Bhaskar


&nbsp;


On Thu, Nov 28, 2019 at 11:36 AM 曾祥才 <xcz200706@qq.com&gt; wrote:

/flink/checkpoints&nbsp; is a external persistent store (a nas directory mounts to the job manager)








------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:29
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"user"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Following are the mandatory condition to run in HA:

a) You should have persistent common external store for jobmanager and task managers to while writing the state
b) You should have persistent external store for zookeeper to store the Jobgraph.


Zookeeper is referring&nbsp; path: /flink/checkpoints/submittedJobGraph480ddf9572ed&nbsp; to get the job graph but jobmanager unable to find it.
It seems /flink/checkpoints&nbsp; is not the external persistent store




Regards
Bhaskar


On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xcz200706@qq.com&gt; wrote:

hi ,I've the same problem with flink 1.9.1 , any solution to fix it
 when the k8s redoploy jobmanager ,&nbsp; the error looks like (seems zk not
 remove submitted job info, but jobmanager remove the file):&nbsp; 
 
 
 Caused by: org.apache.flink.util.FlinkException: Could not retrieve
 submitted JobGraph from state handle under
 /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
 handle is broken. Try cleaning the state handle store.
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
 &nbsp; &nbsp; &nbsp; &nbsp; ... 9 more
 Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain
 block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
 
 
 
 --
 Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

Re: JobGraphs not cleaned up in HA mode

Posted by vino yang <ya...@gmail.com>.
Hi,

Why do you not use HDFS directly?

Best,
Vino

曾祥才 <xc...@qq.com> 于2019年11月28日周四 下午6:48写道:

>
> anyone have the same problem? pls help, thks
>
>
>
> ------------------ 原始邮件 ------------------
> *发件人:* "曾祥才"<xc...@qq.com>;
> *发送时间:* 2019年11月28日(星期四) 下午2:46
> *收件人:* "Vijay Bhaskar"<bh...@gmail.com>;
> *抄送:* "User-Flink"<us...@flink.apache.org>;
> *主题:* 回复: JobGraphs not cleaned up in HA mode
>
> the config  (/flink is the NASdirectory ):
>
> jobmanager.rpc.address: flink-jobmanager
> taskmanager.numberOfTaskSlots: 16
> web.upload.dir: /flink/webUpload
> blob.server.port: 6124
> jobmanager.rpc.port: 6123
> taskmanager.rpc.port: 6122
> jobmanager.heap.size: 1024m
> taskmanager.heap.size: 1024m
> high-availability: zookeeper
> high-availability.cluster-id: /cluster-test
> high-availability.storageDir: /flink/ha
> high-availability.zookeeper.quorum: ****:2181
> high-availability.jobmanager.port: 6123
> high-availability.zookeeper.path.root: /flink/risk-insight
> high-availability.zookeeper.path.checkpoints: /flink/zk-checkpoints
> state.backend: filesystem
> state.checkpoints.dir: file:///flink/checkpoints
> state.savepoints.dir: file:///flink/savepoints
> state.checkpoints.num-retained: 2
> jobmanager.execution.failover-strategy: region
> jobmanager.archive.fs.dir: file:///flink/archive/history
>
>
>
> ------------------ 原始邮件 ------------------
> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
> *发送时间:* 2019年11月28日(星期四) 下午3:12
> *收件人:* "曾祥才"<xc...@qq.com>;
> *抄送:* "User-Flink"<us...@flink.apache.org>;
> *主题:* Re: JobGraphs not cleaned up in HA mode
>
> Can you share the flink configuration once?
>
> Regards
> Bhaskar
>
> On Thu, Nov 28, 2019 at 12:09 PM 曾祥才 <xc...@qq.com> wrote:
>
>> if i clean the zookeeper data , it runs fine .  but next time when the
>> jobmanager failed and redeploy the error occurs again
>>
>>
>>
>>
>> ------------------ 原始邮件 ------------------
>> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
>> *发送时间:* 2019年11月28日(星期四) 下午3:05
>> *收件人:* "曾祥才"<xc...@qq.com>;
>> *主题:* Re: JobGraphs not cleaned up in HA mode
>>
>> Again it could not find the state store file: "Caused by:
>> java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 "
>>  Check why its unable to find.
>> Better thing is: Clean up zookeeper state and check your configurations,
>> correct them and restart cluster.
>> Otherwise it always picks up corrupted state from zookeeper and it will
>> never restart
>>
>> Regards
>> Bhaskar
>>
>> On Thu, Nov 28, 2019 at 11:51 AM 曾祥才 <xc...@qq.com> wrote:
>>
>>> i've made a misstake( the log before is another cluster) . the full
>>> exception log is :
>>>
>>>
>>> INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher      -
>>> Recovering all persisted jobs.
>>> 2019-11-28 02:33:12,726 INFO
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl  -
>>> Starting the SlotManager.
>>> 2019-11-28 02:33:12,743 INFO
>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>> Released locks of job graph 639170a9d710bacfd113ca66b2aacefa from
>>> ZooKeeper.
>>> 2019-11-28 02:33:12,744 ERROR
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Fatal error
>>> occurred in the cluster entrypoint.
>>> org.apache.flink.runtime.dispatcher.DispatcherException: Failed to take
>>> leadership with session id 607e1fe0-f1b4-4af0-af16-4137d779defb.
>>> at
>>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$30(Dispatcher.java:915)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575)
>>> at
>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:594)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456)
>>>
>>> at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
>>> at
>>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44)
>>>
>>> at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>>> at
>>> akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>>>
>>> at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>>> at
>>> akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
>>>
>>> Caused by: java.lang.RuntimeException:
>>> org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph
>>> from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates
>>> that the retrieved state handle is broken. Try cleaning the state handle
>>> store.
>>> at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
>>> at
>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:75)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)
>>> at
>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)
>>>
>>> ... 7 more
>>> Caused by: org.apache.flink.util.FlinkException: Could not retrieve
>>> submitted JobGraph from state handle under
>>> /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state
>>> handle is broken. Try cleaning the state handle store.
>>> at
>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:190)
>>>
>>> at
>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:755)
>>>
>>> at
>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:740)
>>>
>>> at
>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:721)
>>>
>>> at
>>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$27(Dispatcher.java:888)
>>>
>>> at
>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:73)
>>>
>>> ... 9 more
>>> Caused by: java.io.FileNotFoundException:
>>> /flink/ha/submittedJobGraph0c6bcff01199 (No such file or directory)
>>> at java.io.FileInputStream.open0(Native Method)
>>> at java.io.FileInputStream.open(FileInputStream.java:195)
>>>
>>>
>>>
>>>
>>> ------------------ 原始邮件 ------------------
>>> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
>>> *发送时间:* 2019年11月28日(星期四) 下午2:46
>>> *收件人:* "曾祥才"<xc...@qq.com>;
>>> *抄送:* "User-Flink"<us...@flink.apache.org>;
>>> *主题:* Re: JobGraphs not cleaned up in HA mode
>>>
>>> Is it filesystem or hadoop? If its NAS then why the exception "Caused
>>> by: org.apache.hadoop.hdfs.BlockMissingException: "
>>> It seems you configured hadoop state store and giving NAS mount.
>>>
>>> Regards
>>> Bhaskar
>>>
>>>
>>>
>>> On Thu, Nov 28, 2019 at 11:36 AM 曾祥才 <xc...@qq.com> wrote:
>>>
>>>> /flink/checkpoints  is a external persistent store (a nas directory
>>>> mounts to the job manager)
>>>>
>>>>
>>>>
>>>>
>>>> ------------------ 原始邮件 ------------------
>>>> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
>>>> *发送时间:* 2019年11月28日(星期四) 下午2:29
>>>> *收件人:* "曾祥才"<xc...@qq.com>;
>>>> *抄送:* "user"<us...@flink.apache.org>;
>>>> *主题:* Re: JobGraphs not cleaned up in HA mode
>>>>
>>>> Following are the mandatory condition to run in HA:
>>>>
>>>> a) You should have persistent common external store for jobmanager and
>>>> task managers to while writing the state
>>>> b) You should have persistent external store for zookeeper to store the
>>>> Jobgraph.
>>>>
>>>> Zookeeper is referring  path:
>>>> /flink/checkpoints/submittedJobGraph480ddf9572ed  to get the job graph but
>>>> jobmanager unable to find it.
>>>> It seems /flink/checkpoints  is not the external persistent store
>>>>
>>>>
>>>> Regards
>>>> Bhaskar
>>>>
>>>> On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xc...@qq.com> wrote:
>>>>
>>>>> hi ,I've the same problem with flink 1.9.1 , any solution to fix it
>>>>> when the k8s redoploy jobmanager ,  the error looks like (seems zk not
>>>>> remove submitted job info, but jobmanager remove the file):
>>>>>
>>>>>
>>>>> Caused by: org.apache.flink.util.FlinkException: Could not retrieve
>>>>> submitted JobGraph from state handle under
>>>>> /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved
>>>>> state
>>>>> handle is broken. Try cleaning the state handle store.
>>>>>         at
>>>>>
>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
>>>>>         at
>>>>>
>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
>>>>>         at
>>>>>
>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
>>>>>         at
>>>>>
>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
>>>>>         at
>>>>>
>>>>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
>>>>>         at
>>>>>
>>>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
>>>>>         ... 9 more
>>>>> Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not
>>>>> obtain
>>>>> block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
>>>>> file=/flink/checkpoints/submittedJobGraph480ddf9572ed
>>>>>         at
>>>>>
>>>>> org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> Sent from:
>>>>> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
>>>>>
>>>>

回复: JobGraphs not cleaned up in HA mode

Posted by 曾祥才 <xc...@qq.com>.
anyone have the same problem? pls help, thks






------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:46
收件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;回复: JobGraphs not cleaned up in HA mode



the config&nbsp; (/flink is the NASdirectory ):&nbsp; 


jobmanager.rpc.address: flink-jobmanager
taskmanager.numberOfTaskSlots: 16
web.upload.dir: /flink/webUpload
blob.server.port: 6124
jobmanager.rpc.port: 6123
taskmanager.rpc.port: 6122
jobmanager.heap.size: 1024m
taskmanager.heap.size: 1024m
high-availability: zookeeper
high-availability.cluster-id: /cluster-test
high-availability.storageDir: /flink/ha
high-availability.zookeeper.quorum: ****:2181
high-availability.jobmanager.port: 6123
high-availability.zookeeper.path.root: /flink/risk-insight
high-availability.zookeeper.path.checkpoints: /flink/zk-checkpoints
state.backend: filesystem
state.checkpoints.dir: file:///flink/checkpoints
state.savepoints.dir: file:///flink/savepoints
state.checkpoints.num-retained: 2
jobmanager.execution.failover-strategy: region
jobmanager.archive.fs.dir: file:///flink/archive/history

&nbsp;




------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午3:12
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Can you share the flink configuration once?

Regards
Bhaskar


On Thu, Nov 28, 2019 at 12:09 PM 曾祥才 <xcz200706@qq.com&gt; wrote:

if i clean the zookeeper data , it runs fine .&nbsp; but next time when the jobmanager failed and redeploy the error occurs again








------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午3:05
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;

主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Again it could not find the state store file: "Caused by: java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 "&nbsp;Check why its unable to find. 
Better thing is: Clean up zookeeper state and check your configurations, correct them and restart cluster.
Otherwise it always picks up corrupted state from zookeeper and it will never restart


Regards
Bhaskar


On Thu, Nov 28, 2019 at 11:51 AM 曾祥才 <xcz200706@qq.com&gt; wrote:

i've made a misstake( the log before is another cluster) . the full exception log is :


INFO&nbsp; org.apache.flink.runtime.dispatcher.StandaloneDispatcher&nbsp; &nbsp; &nbsp; - Recovering all persisted jobs. 
2019-11-28 02:33:12,726 INFO&nbsp; org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl&nbsp; - Starting the SlotManager. 
2019-11-28 02:33:12,743 INFO&nbsp; org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore&nbsp; - Released locks of job graph 639170a9d710bacfd113ca66b2aacefa from ZooKeeper. 
2019-11-28 02:33:12,744 ERROR org.apache.flink.runtime.entrypoint.ClusterEntrypoint&nbsp; &nbsp; &nbsp; &nbsp; &nbsp;- Fatal error occurred in the cluster entrypoint. 
org.apache.flink.runtime.dispatcher.DispatcherException: Failed to take leadership with session id 607e1fe0-f1b4-4af0-af16-4137d779defb. 
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$30(Dispatcher.java:915) 
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774) 
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750) 
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) 
	at java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575) 
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:594) 
	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456) 
	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40) 
	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44) 
	at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) 
	at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) 
	at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) 
	at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) 
Caused by: java.lang.RuntimeException: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state handle is broken. Try cleaning the state handle store. 
	at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199) 
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:75) 
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616) 
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591) 
	... 7 more 
Caused by: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state handle is broken. Try cleaning the state handle store. 
	at org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:190) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:755) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:740) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:721) 
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$27(Dispatcher.java:888) 
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:73) 
	... 9 more 
Caused by: java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 (No such file or directory) 
	at java.io.FileInputStream.open0(Native Method) 
	at java.io.FileInputStream.open(FileInputStream.java:195) 



&nbsp;




------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:46
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Is it filesystem or hadoop? If its NAS then why the exception "Caused by: org.apache.hadoop.hdfs.BlockMissingException: "It seems you configured hadoop state store and giving NAS mount. 


Regards
Bhaskar


&nbsp;


On Thu, Nov 28, 2019 at 11:36 AM 曾祥才 <xcz200706@qq.com&gt; wrote:

/flink/checkpoints&nbsp; is a external persistent store (a nas directory mounts to the job manager)








------------------ 原始邮件 ------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:29
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"user"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Following are the mandatory condition to run in HA:

a) You should have persistent common external store for jobmanager and task managers to while writing the state
b) You should have persistent external store for zookeeper to store the Jobgraph.


Zookeeper is referring&nbsp; path: /flink/checkpoints/submittedJobGraph480ddf9572ed&nbsp; to get the job graph but jobmanager unable to find it.
It seems /flink/checkpoints&nbsp; is not the external persistent store




Regards
Bhaskar


On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xcz200706@qq.com&gt; wrote:

hi ,I've the same problem with flink 1.9.1 , any solution to fix it
 when the k8s redoploy jobmanager ,&nbsp; the error looks like (seems zk not
 remove submitted job info, but jobmanager remove the file):&nbsp; 
 
 
 Caused by: org.apache.flink.util.FlinkException: Could not retrieve
 submitted JobGraph from state handle under
 /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
 handle is broken. Try cleaning the state handle store.
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
 &nbsp; &nbsp; &nbsp; &nbsp; ... 9 more
 Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain
 block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
 
 
 
 --
 Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

回复: JobGraphs not cleaned up in HA mode

Posted by 曾祥才 <xc...@qq.com>.
the config&nbsp; (/flink is the NASdirectory ):&nbsp;&nbsp;


jobmanager.rpc.address: flink-jobmanager
taskmanager.numberOfTaskSlots: 16
web.upload.dir: /flink/webUpload
blob.server.port: 6124
jobmanager.rpc.port: 6123
taskmanager.rpc.port: 6122
jobmanager.heap.size: 1024m
taskmanager.heap.size: 1024m
high-availability: zookeeper
high-availability.cluster-id: /cluster-test
high-availability.storageDir: /flink/ha
high-availability.zookeeper.quorum: ****:2181
high-availability.jobmanager.port: 6123
high-availability.zookeeper.path.root: /flink/risk-insight
high-availability.zookeeper.path.checkpoints: /flink/zk-checkpoints
state.backend: filesystem
state.checkpoints.dir: file:///flink/checkpoints
state.savepoints.dir: file:///flink/savepoints
state.checkpoints.num-retained: 2
jobmanager.execution.failover-strategy: region
jobmanager.archive.fs.dir: file:///flink/archive/history

&nbsp;




------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午3:12
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Can you share the flink configuration&nbsp;once?

Regards
Bhaskar


On Thu, Nov 28, 2019 at 12:09 PM 曾祥才 <xcz200706@qq.com&gt; wrote:

if i clean the zookeeper data , it runs fine .&nbsp; but next time when the jobmanager failed and redeploy the error occurs again








------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午3:05
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;

主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Again it could not find the state store file: "Caused by: java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199&nbsp;"&nbsp;Check why its unable to find.&nbsp;
Better thing is: Clean up zookeeper state and check your configurations, correct them and restart cluster.
Otherwise it always&nbsp;picks up corrupted state from zookeeper and it will never restart


Regards
Bhaskar


On Thu, Nov 28, 2019 at 11:51 AM 曾祥才 <xcz200706@qq.com&gt; wrote:

i've made a misstake( the log before is another cluster) . the full exception log is :


INFO&nbsp; org.apache.flink.runtime.dispatcher.StandaloneDispatcher&nbsp; &nbsp; &nbsp; - Recovering all persisted jobs.&nbsp;
2019-11-28 02:33:12,726 INFO&nbsp; org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl&nbsp; - Starting the SlotManager.&nbsp;
2019-11-28 02:33:12,743 INFO&nbsp; org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore&nbsp; - Released locks of job graph 639170a9d710bacfd113ca66b2aacefa from ZooKeeper.&nbsp;
2019-11-28 02:33:12,744 ERROR org.apache.flink.runtime.entrypoint.ClusterEntrypoint&nbsp; &nbsp; &nbsp; &nbsp; &nbsp;- Fatal error occurred in the cluster entrypoint.&nbsp;
org.apache.flink.runtime.dispatcher.DispatcherException: Failed to take leadership with session id 607e1fe0-f1b4-4af0-af16-4137d779defb.&nbsp;
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$30(Dispatcher.java:915)&nbsp;
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)&nbsp;
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)&nbsp;
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)&nbsp;
	at java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575)&nbsp;
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:594)&nbsp;
	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456)&nbsp;
	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)&nbsp;
	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44)&nbsp;
	at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)&nbsp;
	at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)&nbsp;
	at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)&nbsp;
	at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)&nbsp;
Caused by: java.lang.RuntimeException: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state handle is broken. Try cleaning the state handle store.&nbsp;
	at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)&nbsp;
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:75)&nbsp;
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)&nbsp;
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)&nbsp;
	... 7 more&nbsp;
Caused by: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state handle is broken. Try cleaning the state handle store.&nbsp;
	at org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:190)&nbsp;
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:755)&nbsp;
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:740)&nbsp;
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:721)&nbsp;
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$27(Dispatcher.java:888)&nbsp;
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:73)&nbsp;
	... 9 more&nbsp;
Caused by: java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 (No such file or directory)&nbsp;
	at java.io.FileInputStream.open0(Native Method)&nbsp;
	at java.io.FileInputStream.open(FileInputStream.java:195)&nbsp;



&nbsp;




------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:46
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Is it filesystem or hadoop? If its NAS then why the exception "Caused by: org.apache.hadoop.hdfs.BlockMissingException:&nbsp;"It seems you configured hadoop state store and giving NAS mount.&nbsp;


Regards
Bhaskar


&nbsp;


On Thu, Nov 28, 2019 at 11:36 AM 曾祥才 <xcz200706@qq.com&gt; wrote:

/flink/checkpoints&nbsp; is a external persistent store&nbsp;(a nas directory mounts to the job manager)








------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:29
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"user"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Following are the mandatory condition to run in HA:

a) You should have persistent common external store for jobmanager and task managers to while writing the state
b) You should have persistent external store for zookeeper to store the Jobgraph.


Zookeeper is referring&nbsp; path: /flink/checkpoints/submittedJobGraph480ddf9572ed&nbsp; to get the job graph but jobmanager unable to find it.
It seems /flink/checkpoints&nbsp; is not the external persistent store




Regards
Bhaskar


On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xcz200706@qq.com&gt; wrote:

hi ,I've the same problem with flink 1.9.1 , any solution to fix it
 when the k8s redoploy jobmanager ,&nbsp; the error looks like (seems zk not
 remove submitted job info, but jobmanager remove the file):&nbsp; 
 
 
 Caused by: org.apache.flink.util.FlinkException: Could not retrieve
 submitted JobGraph from state handle under
 /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
 handle is broken. Try cleaning the state handle store.
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
 &nbsp; &nbsp; &nbsp; &nbsp; ... 9 more
 Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain
 block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
 
 
 
 --
 Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

Re: JobGraphs not cleaned up in HA mode

Posted by Vijay Bhaskar <bh...@gmail.com>.
Can you share the flink configuration once?

Regards
Bhaskar

On Thu, Nov 28, 2019 at 12:09 PM 曾祥才 <xc...@qq.com> wrote:

> if i clean the zookeeper data , it runs fine .  but next time when the
> jobmanager failed and redeploy the error occurs again
>
>
>
>
> ------------------ 原始邮件 ------------------
> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
> *发送时间:* 2019年11月28日(星期四) 下午3:05
> *收件人:* "曾祥才"<xc...@qq.com>;
> *主题:* Re: JobGraphs not cleaned up in HA mode
>
> Again it could not find the state store file: "Caused by:
> java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 "
>  Check why its unable to find.
> Better thing is: Clean up zookeeper state and check your configurations,
> correct them and restart cluster.
> Otherwise it always picks up corrupted state from zookeeper and it will
> never restart
>
> Regards
> Bhaskar
>
> On Thu, Nov 28, 2019 at 11:51 AM 曾祥才 <xc...@qq.com> wrote:
>
>> i've made a misstake( the log before is another cluster) . the full
>> exception log is :
>>
>>
>> INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher      -
>> Recovering all persisted jobs.
>> 2019-11-28 02:33:12,726 INFO
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl  -
>> Starting the SlotManager.
>> 2019-11-28 02:33:12,743 INFO
>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>> Released locks of job graph 639170a9d710bacfd113ca66b2aacefa from
>> ZooKeeper.
>> 2019-11-28 02:33:12,744 ERROR
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Fatal error
>> occurred in the cluster entrypoint.
>> org.apache.flink.runtime.dispatcher.DispatcherException: Failed to take
>> leadership with session id 607e1fe0-f1b4-4af0-af16-4137d779defb.
>> at
>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$30(Dispatcher.java:915)
>> at
>> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
>> at
>> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
>> at
>> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
>> at
>> java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575)
>> at
>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:594)
>> at
>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456)
>> at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
>> at
>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44)
>> at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>> at
>> akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>> at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>> at
>> akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
>> Caused by: java.lang.RuntimeException:
>> org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph
>> from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates
>> that the retrieved state handle is broken. Try cleaning the state handle
>> store.
>> at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
>> at
>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:75)
>> at
>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)
>> at
>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)
>> ... 7 more
>> Caused by: org.apache.flink.util.FlinkException: Could not retrieve
>> submitted JobGraph from state handle under
>> /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state
>> handle is broken. Try cleaning the state handle store.
>> at
>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:190)
>> at
>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:755)
>> at
>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:740)
>> at
>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:721)
>> at
>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$27(Dispatcher.java:888)
>> at
>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:73)
>> ... 9 more
>> Caused by: java.io.FileNotFoundException:
>> /flink/ha/submittedJobGraph0c6bcff01199 (No such file or directory)
>> at java.io.FileInputStream.open0(Native Method)
>> at java.io.FileInputStream.open(FileInputStream.java:195)
>>
>>
>>
>>
>> ------------------ 原始邮件 ------------------
>> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
>> *发送时间:* 2019年11月28日(星期四) 下午2:46
>> *收件人:* "曾祥才"<xc...@qq.com>;
>> *抄送:* "User-Flink"<us...@flink.apache.org>;
>> *主题:* Re: JobGraphs not cleaned up in HA mode
>>
>> Is it filesystem or hadoop? If its NAS then why the exception "Caused by:
>> org.apache.hadoop.hdfs.BlockMissingException: "
>> It seems you configured hadoop state store and giving NAS mount.
>>
>> Regards
>> Bhaskar
>>
>>
>>
>> On Thu, Nov 28, 2019 at 11:36 AM 曾祥才 <xc...@qq.com> wrote:
>>
>>> /flink/checkpoints  is a external persistent store (a nas directory
>>> mounts to the job manager)
>>>
>>>
>>>
>>>
>>> ------------------ 原始邮件 ------------------
>>> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
>>> *发送时间:* 2019年11月28日(星期四) 下午2:29
>>> *收件人:* "曾祥才"<xc...@qq.com>;
>>> *抄送:* "user"<us...@flink.apache.org>;
>>> *主题:* Re: JobGraphs not cleaned up in HA mode
>>>
>>> Following are the mandatory condition to run in HA:
>>>
>>> a) You should have persistent common external store for jobmanager and
>>> task managers to while writing the state
>>> b) You should have persistent external store for zookeeper to store the
>>> Jobgraph.
>>>
>>> Zookeeper is referring  path:
>>> /flink/checkpoints/submittedJobGraph480ddf9572ed  to get the job graph but
>>> jobmanager unable to find it.
>>> It seems /flink/checkpoints  is not the external persistent store
>>>
>>>
>>> Regards
>>> Bhaskar
>>>
>>> On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xc...@qq.com> wrote:
>>>
>>>> hi ,I've the same problem with flink 1.9.1 , any solution to fix it
>>>> when the k8s redoploy jobmanager ,  the error looks like (seems zk not
>>>> remove submitted job info, but jobmanager remove the file):
>>>>
>>>>
>>>> Caused by: org.apache.flink.util.FlinkException: Could not retrieve
>>>> submitted JobGraph from state handle under
>>>> /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved
>>>> state
>>>> handle is broken. Try cleaning the state handle store.
>>>>         at
>>>>
>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
>>>>         at
>>>>
>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
>>>>         at
>>>>
>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
>>>>         at
>>>>
>>>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
>>>>         at
>>>>
>>>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
>>>>         at
>>>>
>>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
>>>>         ... 9 more
>>>> Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not
>>>> obtain
>>>> block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
>>>> file=/flink/checkpoints/submittedJobGraph480ddf9572ed
>>>>         at
>>>>
>>>> org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
>>>>
>>>>
>>>>
>>>> --
>>>> Sent from:
>>>> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
>>>>
>>>

回复: JobGraphs not cleaned up in HA mode

Posted by 曾祥才 <xc...@qq.com>.
if i clean the zookeeper data , it runs fine .&nbsp; but next time when the jobmanager failed and redeploy the error occurs again








------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午3:05
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;

主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Again it could not find the state store file: "Caused by: java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199&nbsp;"&nbsp;Check why its unable to find.&nbsp;
Better thing is: Clean up zookeeper state and check your configurations, correct them and restart cluster.
Otherwise it always&nbsp;picks up corrupted state from zookeeper and it will never restart


Regards
Bhaskar


On Thu, Nov 28, 2019 at 11:51 AM 曾祥才 <xcz200706@qq.com&gt; wrote:

i've made a misstake( the log before is another cluster) . the full exception log is :


INFO&nbsp; org.apache.flink.runtime.dispatcher.StandaloneDispatcher&nbsp; &nbsp; &nbsp; - Recovering all persisted jobs.&nbsp;
2019-11-28 02:33:12,726 INFO&nbsp; org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl&nbsp; - Starting the SlotManager.&nbsp;
2019-11-28 02:33:12,743 INFO&nbsp; org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore&nbsp; - Released locks of job graph 639170a9d710bacfd113ca66b2aacefa from ZooKeeper.&nbsp;
2019-11-28 02:33:12,744 ERROR org.apache.flink.runtime.entrypoint.ClusterEntrypoint&nbsp; &nbsp; &nbsp; &nbsp; &nbsp;- Fatal error occurred in the cluster entrypoint.&nbsp;
org.apache.flink.runtime.dispatcher.DispatcherException: Failed to take leadership with session id 607e1fe0-f1b4-4af0-af16-4137d779defb.&nbsp;
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$30(Dispatcher.java:915)&nbsp;
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)&nbsp;
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)&nbsp;
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)&nbsp;
	at java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575)&nbsp;
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:594)&nbsp;
	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456)&nbsp;
	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)&nbsp;
	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44)&nbsp;
	at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)&nbsp;
	at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)&nbsp;
	at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)&nbsp;
	at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)&nbsp;
Caused by: java.lang.RuntimeException: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state handle is broken. Try cleaning the state handle store.&nbsp;
	at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)&nbsp;
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:75)&nbsp;
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)&nbsp;
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)&nbsp;
	... 7 more&nbsp;
Caused by: org.apache.flink.util.FlinkException: Could not retrieve submitted JobGraph from state handle under /639170a9d710bacfd113ca66b2aacefa. This indicates that the retrieved state handle is broken. Try cleaning the state handle store.&nbsp;
	at org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:190)&nbsp;
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:755)&nbsp;
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:740)&nbsp;
	at org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:721)&nbsp;
	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$27(Dispatcher.java:888)&nbsp;
	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:73)&nbsp;
	... 9 more&nbsp;
Caused by: java.io.FileNotFoundException: /flink/ha/submittedJobGraph0c6bcff01199 (No such file or directory)&nbsp;
	at java.io.FileInputStream.open0(Native Method)&nbsp;
	at java.io.FileInputStream.open(FileInputStream.java:195)&nbsp;



&nbsp;




------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:46
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"User-Flink"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Is it filesystem or hadoop? If its NAS then why the exception "Caused by: org.apache.hadoop.hdfs.BlockMissingException:&nbsp;"It seems you configured hadoop state store and giving NAS mount.&nbsp;


Regards
Bhaskar


&nbsp;


On Thu, Nov 28, 2019 at 11:36 AM 曾祥才 <xcz200706@qq.com&gt; wrote:

/flink/checkpoints&nbsp; is a external persistent store&nbsp;(a nas directory mounts to the job manager)








------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:29
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"user"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Following are the mandatory condition to run in HA:

a) You should have persistent common external store for jobmanager and task managers to while writing the state
b) You should have persistent external store for zookeeper to store the Jobgraph.


Zookeeper is referring&nbsp; path: /flink/checkpoints/submittedJobGraph480ddf9572ed&nbsp; to get the job graph but jobmanager unable to find it.
It seems /flink/checkpoints&nbsp; is not the external persistent store




Regards
Bhaskar


On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xcz200706@qq.com&gt; wrote:

hi ,I've the same problem with flink 1.9.1 , any solution to fix it
 when the k8s redoploy jobmanager ,&nbsp; the error looks like (seems zk not
 remove submitted job info, but jobmanager remove the file):&nbsp; 
 
 
 Caused by: org.apache.flink.util.FlinkException: Could not retrieve
 submitted JobGraph from state handle under
 /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
 handle is broken. Try cleaning the state handle store.
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
 &nbsp; &nbsp; &nbsp; &nbsp; ... 9 more
 Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain
 block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
 
 
 
 --
 Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

Re: JobGraphs not cleaned up in HA mode

Posted by Vijay Bhaskar <bh...@gmail.com>.
Is it filesystem or hadoop? If its NAS then why the exception "Caused by:
org.apache.hadoop.hdfs.BlockMissingException: "
It seems you configured hadoop state store and giving NAS mount.

Regards
Bhaskar



On Thu, Nov 28, 2019 at 11:36 AM 曾祥才 <xc...@qq.com> wrote:

> /flink/checkpoints  is a external persistent store (a nas directory mounts
> to the job manager)
>
>
>
>
> ------------------ 原始邮件 ------------------
> *发件人:* "Vijay Bhaskar"<bh...@gmail.com>;
> *发送时间:* 2019年11月28日(星期四) 下午2:29
> *收件人:* "曾祥才"<xc...@qq.com>;
> *抄送:* "user"<us...@flink.apache.org>;
> *主题:* Re: JobGraphs not cleaned up in HA mode
>
> Following are the mandatory condition to run in HA:
>
> a) You should have persistent common external store for jobmanager and
> task managers to while writing the state
> b) You should have persistent external store for zookeeper to store the
> Jobgraph.
>
> Zookeeper is referring  path:
> /flink/checkpoints/submittedJobGraph480ddf9572ed  to get the job graph but
> jobmanager unable to find it.
> It seems /flink/checkpoints  is not the external persistent store
>
>
> Regards
> Bhaskar
>
> On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xc...@qq.com> wrote:
>
>> hi ,I've the same problem with flink 1.9.1 , any solution to fix it
>> when the k8s redoploy jobmanager ,  the error looks like (seems zk not
>> remove submitted job info, but jobmanager remove the file):
>>
>>
>> Caused by: org.apache.flink.util.FlinkException: Could not retrieve
>> submitted JobGraph from state handle under
>> /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
>> handle is broken. Try cleaning the state handle store.
>>         at
>>
>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
>>         at
>>
>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
>>         at
>>
>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
>>         at
>>
>> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
>>         at
>>
>> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
>>         at
>>
>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
>>         ... 9 more
>> Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain
>> block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
>> file=/flink/checkpoints/submittedJobGraph480ddf9572ed
>>         at
>>
>> org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
>>
>>
>>
>> --
>> Sent from:
>> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
>>
>

回复: JobGraphs not cleaned up in HA mode

Posted by 曾祥才 <xc...@qq.com>.
/flink/checkpoints&nbsp; is a external persistent store&nbsp;(a nas directory mounts to the job manager)








------------------&nbsp;原始邮件&nbsp;------------------
发件人:&nbsp;"Vijay Bhaskar"<bhaskar.ebay77@gmail.com&gt;;
发送时间:&nbsp;2019年11月28日(星期四) 下午2:29
收件人:&nbsp;"曾祥才"<xcz200706@qq.com&gt;;
抄送:&nbsp;"user"<user@flink.apache.org&gt;;
主题:&nbsp;Re: JobGraphs not cleaned up in HA mode



Following are the mandatory condition to run in HA:

a) You should have persistent common external store for jobmanager and task managers to while writing the state
b) You should have persistent external store for zookeeper to store the Jobgraph.


Zookeeper is referring&nbsp; path: /flink/checkpoints/submittedJobGraph480ddf9572ed&nbsp; to get the job graph but jobmanager unable to find it.
It seems /flink/checkpoints&nbsp; is not the external persistent store




Regards
Bhaskar


On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xcz200706@qq.com&gt; wrote:

hi ,I've the same problem with flink 1.9.1 , any solution to fix it
 when the k8s redoploy jobmanager ,&nbsp; the error looks like (seems zk not
 remove submitted job info, but jobmanager remove the file):&nbsp; 
 
 
 Caused by: org.apache.flink.util.FlinkException: Could not retrieve
 submitted JobGraph from state handle under
 /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
 handle is broken. Try cleaning the state handle store.
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
 &nbsp; &nbsp; &nbsp; &nbsp; ... 9 more
 Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain
 block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
 file=/flink/checkpoints/submittedJobGraph480ddf9572ed
 &nbsp; &nbsp; &nbsp; &nbsp; at
 org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
 
 
 
 --
 Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

Re: JobGraphs not cleaned up in HA mode

Posted by Vijay Bhaskar <bh...@gmail.com>.
Following are the mandatory condition to run in HA:

a) You should have persistent common external store for jobmanager and task
managers to while writing the state
b) You should have persistent external store for zookeeper to store the
Jobgraph.

Zookeeper is referring  path:
/flink/checkpoints/submittedJobGraph480ddf9572ed  to get the job graph but
jobmanager unable to find it.
It seems /flink/checkpoints  is not the external persistent store


Regards
Bhaskar

On Thu, Nov 28, 2019 at 10:43 AM seuzxc <xc...@qq.com> wrote:

> hi ,I've the same problem with flink 1.9.1 , any solution to fix it
> when the k8s redoploy jobmanager ,  the error looks like (seems zk not
> remove submitted job info, but jobmanager remove the file):
>
>
> Caused by: org.apache.flink.util.FlinkException: Could not retrieve
> submitted JobGraph from state handle under
> /147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
> handle is broken. Try cleaning the state handle store.
>         at
>
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
>         at
>
> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
>         at
>
> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
>         at
>
> org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
>         at
>
> org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
>         at
>
> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
>         ... 9 more
> Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain
> block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
> file=/flink/checkpoints/submittedJobGraph480ddf9572ed
>         at
>
> org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)
>
>
>
> --
> Sent from:
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
>

Re: JobGraphs not cleaned up in HA mode

Posted by seuzxc <xc...@qq.com>.
hi ,I've the same problem with flink 1.9.1 , any solution to fix it
when the k8s redoploy jobmanager ,  the error looks like (seems zk not
remove submitted job info, but jobmanager remove the file):  


Caused by: org.apache.flink.util.FlinkException: Could not retrieve
submitted JobGraph from state handle under
/147dd022ec91f7381ad4ca3d290387e9. This indicates that the retrieved state
handle is broken. Try cleaning the state handle store.
	at
org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:208)
	at
org.apache.flink.runtime.dispatcher.Dispatcher.recoverJob(Dispatcher.java:696)
	at
org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobGraphs(Dispatcher.java:681)
	at
org.apache.flink.runtime.dispatcher.Dispatcher.recoverJobs(Dispatcher.java:662)
	at
org.apache.flink.runtime.dispatcher.Dispatcher.lambda$null$26(Dispatcher.java:821)
	at
org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:72)
	... 9 more
Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain
block: BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
file=/flink/checkpoints/submittedJobGraph480ddf9572ed
	at
org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1052)



--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

Re: JobGraphs not cleaned up in HA mode

Posted by Encho Mishinev <en...@gmail.com>.
Hi Till,

That's great that you've traced the problem. It seems like a lot of people
have been reporting similar problems. Thanks for reacting so quickly and
good luck with fixing the bug. I will use a single JobManager with K8S
Deployment for now, but look forward to the fix.

Thanks,
Encho

On Wed, Aug 29, 2018 at 4:43 PM Till Rohrmann <tr...@apache.org> wrote:

> Hi Encho,
>
> thanks for sending me the logs. I think I found a bug which could explain
> what you are observing: We listen to newly added jobs and try to recover
> them independent of the leadership status. Due to this also a standby
> JobManager tries to recover a submitted job but won't execute it.
> Unfortunately, recovering a job already locks it without releasing the lock
> if it cannot be executed. I've documented the problem here [1]. This is a
> quite mean bug which we should fix asap. Thanks a lot for reporting the
> problem!
>
> [1] https://issues.apache.org/jira/browse/FLINK-10255.
>
> Cheers,
> Till
>
> On Wed, Aug 29, 2018 at 3:31 PM Encho Mishinev <en...@gmail.com>
> wrote:
>
>> Hi Till,
>>
>> Those are actually the full logs except the two parts I shortened
>> (pipeline construction and execution). As I said - accessing the UI for
>> Jobmanager 2 redirects to Jobmanager 1 so it seems like he is aware that he
>> is not the leader. Jobmanager 2 has no other logs than what I sent. Here is
>> the full end-to-end log of Jobmanager 2 after repeating the experiment
>> again:
>>
>> Starting Job Manager
>> sed: cannot rename /opt/flink/conf/sediVa6XS: Device or resource busy
>> config file:
>> jobmanager.rpc.address: flink-jobmanager-2
>> jobmanager.rpc.port: 6123
>> jobmanager.heap.size: 8192
>> taskmanager.heap.size: 8192
>> taskmanager.numberOfTaskSlots: 4
>> high-availability: zookeeper
>> high-availability.storageDir:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>> high-availability.zookeeper.quorum: zk-cs:2181
>> high-availability.zookeeper.path.root: /flink
>> high-availability.jobmanager.port: 50010
>> state.backend: filesystem
>> state.checkpoints.dir:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>> state.savepoints.dir:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>> state.backend.incremental: false
>> fs.default-scheme:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>> rest.port: 8081
>> web.upload.dir: /opt/flink/upload
>> query.server.port: 6125
>> taskmanager.numberOfTaskSlots: 4
>> classloader.parent-first-patterns.additional: org.apache.xerces.
>> blob.storage.directory: /opt/flink/blob-server
>> blob.server.port: 6124
>> blob.server.port: 6124
>> query.server.port: 6125
>> Starting standalonesession as a console application on host
>> flink-jobmanager-2-7844b78c9-zwdqv.
>> 2018-08-29 13:19:24,047 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>> --------------------------------------------------------------------------------
>> 2018-08-29 13:19:24,049 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
>> StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
>> Date:16.08.2018 @ 06:39:50 GMT)
>> 2018-08-29 13:19:24,049 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
>> user: flink
>> 2018-08-29 13:19:24,367 WARN  org.apache.hadoop.util.NativeCodeLoader
>>                    - Unable to load native-hadoop library for your
>> platform... using builtin-java classes where applicable
>> 2018-08-29 13:19:24,431 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
>> Hadoop/Kerberos user: flink
>> 2018-08-29 13:19:24,431 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
>> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
>> 2018-08-29 13:19:24,431 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
>> heap size: 6702 MiBytes
>> 2018-08-29 13:19:24,431 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
>> /docker-java-home/jre
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
>> version: 2.7.5
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
>> Options:
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
>> Arguments:
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  --configDir
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  /opt/flink/conf
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  --executionMode
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
>> 2018-08-29 13:19:24,434 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>> 2018-08-29 13:19:24,435 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
>> /opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>> 2018-08-29 13:19:24,435 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>> --------------------------------------------------------------------------------
>> 2018-08-29 13:19:24,436 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
>> UNIX signal handlers for [TERM, HUP, INT]
>> 2018-08-29 13:19:24,442 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: jobmanager.rpc.address, flink-jobmanager-2
>> 2018-08-29 13:19:24,442 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: jobmanager.rpc.port, 6123
>> 2018-08-29 13:19:24,442 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: jobmanager.heap.size, 8192
>> 2018-08-29 13:19:24,442 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: taskmanager.heap.size, 8192
>> 2018-08-29 13:19:24,442 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: taskmanager.numberOfTaskSlots, 4
>> 2018-08-29 13:19:24,442 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability, zookeeper
>> 2018-08-29 13:19:24,443 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.storageDir,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>> 2018-08-29 13:19:24,443 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.zookeeper.quorum, zk-cs:2181
>> 2018-08-29 13:19:24,443 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.zookeeper.path.root, /flink
>> 2018-08-29 13:19:24,443 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.jobmanager.port, 50010
>> 2018-08-29 13:19:24,443 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.backend, filesystem
>> 2018-08-29 13:19:24,443 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.checkpoints.dir,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>> 2018-08-29 13:19:24,444 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.savepoints.dir,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>> 2018-08-29 13:19:24,444 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.backend.incremental, false
>> 2018-08-29 13:19:24,444 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: fs.default-scheme,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>> 2018-08-29 13:19:24,444 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: rest.port, 8081
>> 2018-08-29 13:19:24,444 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: web.upload.dir, /opt/flink/upload
>> 2018-08-29 13:19:24,444 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: query.server.port, 6125
>> 2018-08-29 13:19:24,445 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: taskmanager.numberOfTaskSlots, 4
>> 2018-08-29 13:19:24,445 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: classloader.parent-first-patterns.additional,
>> org.apache.xerces.
>> 2018-08-29 13:19:24,445 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: blob.storage.directory, /opt/flink/blob-server
>> 2018-08-29 13:19:24,445 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: blob.server.port, 6124
>> 2018-08-29 13:19:24,445 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: blob.server.port, 6124
>> 2018-08-29 13:19:24,445 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: query.server.port, 6125
>> 2018-08-29 13:19:24,461 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
>> StandaloneSessionClusterEntrypoint.
>> 2018-08-29 13:19:24,461 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>> default filesystem.
>> 2018-08-29 13:19:24,472 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>> security context.
>> 2018-08-29 13:19:24,506 INFO
>> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
>> set to flink (auth:SIMPLE)
>> 2018-08-29 13:19:24,522 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>> Initializing cluster services.
>> 2018-08-29 13:19:24,532 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
>> start actor system at flink-jobmanager-2:50010
>> 2018-08-29 13:19:24,996 INFO  akka.event.slf4j.Slf4jLogger
>>                   - Slf4jLogger started
>> 2018-08-29 13:19:25,050 INFO  akka.remote.Remoting
>>                   - Starting remoting
>> 2018-08-29 13:19:25,209 INFO  akka.remote.Remoting
>>                   - Remoting started; listening on addresses
>> :[akka.tcp://flink@flink-jobmanager-2:50010]
>> 2018-08-29 13:19:25,216 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
>> system started at akka.tcp://flink@flink-jobmanager-2:50010
>> 2018-08-29 13:19:25,648 INFO
>> org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
>> highly available BLOB storage directory at
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
>> 2018-08-29 13:19:25,702 INFO
>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing
>> default ACL for ZK connections
>> 2018-08-29 13:19:25,703 INFO
>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Using
>> '/flink/default' as Zookeeper namespace.
>> 2018-08-29 13:19:25,750 INFO
>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
>> - Starting
>> 2018-08-29 13:19:25,756 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
>> built on 03/23/2017 10:13 GMT
>> 2018-08-29 13:19:25,756 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:host.name=flink-jobmanager-2-7844b78c9-zwdqv
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.version=1.8.0_181
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.vendor=Oracle Corporation
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.io.tmpdir=/tmp
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.compiler=<NA>
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:os.name=Linux
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:os.arch=amd64
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:os.version=4.4.0-1027-gke
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:user.name=flink
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:user.home=/opt/flink
>> 2018-08-29 13:19:25,757 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:user.dir=/opt/flink
>> 2018-08-29 13:19:25,758 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
>> Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
>> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
>> 2018-08-29 13:19:25,775 WARN
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
>> configuration failed: javax.security.auth.login.LoginException: No JAAS
>> configuration section named 'Client' was found in specified JAAS
>> configuration file: '/tmp/jaas-5000339768628554676.conf'. Will continue
>> connection to Zookeeper server without SASL authentication, if Zookeeper
>> server allows it.
>> 2018-08-29 13:19:25,776 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>> Opening socket connection to server zk-cs.default.svc.cluster.local/
>> 10.27.248.104:2181
>> 2018-08-29 13:19:25,777 ERROR
>> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
>> Authentication failed
>> 2018-08-29 13:19:25,777 INFO  org.apache.flink.runtime.blob.BlobServer
>>                   - Created BLOB server storage directory
>> /opt/flink/blob-server/blobStore-40cefeee-e8d1-4522-aea3-957d9f7fbeee
>> 2018-08-29 13:19:25,777 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
>> connection established to zk-cs.default.svc.cluster.local/
>> 10.27.248.104:2181, initiating session
>> 2018-08-29 13:19:25,778 INFO  org.apache.flink.runtime.blob.BlobServer
>>                   - Started BLOB server at 0.0.0.0:6124 - max concurrent
>> requests: 50 - max backlog: 1000
>> 2018-08-29 13:19:25,788 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>> Session establishment complete on server zk-cs.default.svc.cluster.local/
>> 10.27.248.104:2181, sessionid = 0x26584fd55690009, negotiated timeout =
>> 40000
>> 2018-08-29 13:19:25,789 INFO
>> org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
>> - State change: CONNECTED
>> 2018-08-29 13:19:25,793 INFO
>> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
>> reporter configured, no metrics will be exposed/reported.
>> 2018-08-29 13:19:25,798 INFO
>> org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
>> Initializing FileArchivedExecutionGraphStore: Storage directory
>> /tmp/executionGraphStore-76cce4e7-84ea-4624-a847-bbd7fdc4f109, expiration
>> time 3600000, maximum cache size 52428800 bytes.
>> 2018-08-29 13:19:25,824 INFO
>> org.apache.flink.runtime.blob.TransientBlobCache              - Created
>> BLOB cache storage directory
>> /opt/flink/blob-server/blobStore-7c6c2db0-f7ab-4cb6-909d-6c9cbfd78215
>> 2018-08-29 13:19:25,838 WARN
>> org.apache.flink.configuration.Configuration                  - Config uses
>> deprecated configuration key 'jobmanager.rpc.address' instead of proper key
>> 'rest.address'
>> 2018-08-29 13:19:25,839 WARN
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
>> directory /opt/flink/upload/flink-web-upload does not exist, or has been
>> deleted externally. Previously uploaded files are no longer available.
>> 2018-08-29 13:19:25,840 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
>> directory /opt/flink/upload/flink-web-upload for file uploads.
>> 2018-08-29 13:19:25,843 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
>> rest endpoint.
>> 2018-08-29 13:19:26,143 WARN
>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
>> environment variable 'log.file' is not set.
>> 2018-08-29 13:19:26,143 WARN
>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
>> log files are unavailable in the web dashboard. Log file location not found
>> in environment variable 'log.file' or configuration key 'Key:
>> 'web.log.path' , default: null (deprecated keys:
>> [jobmanager.web.log.path])'.
>> 2018-08-29 13:19:26,216 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
>> endpoint listening at flink-jobmanager-2:8081
>> 2018-08-29 13:19:26,217 INFO
>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>> Starting ZooKeeperLeaderElectionService
>> ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
>> 2018-08-29 13:19:26,236 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
>> frontend listening at http://flink-jobmanager-2:8081.
>> 2018-08-29 13:19:26,248 INFO
>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>> RPC endpoint for
>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
>> akka://flink/user/resourcemanager .
>> 2018-08-29 13:19:26,323 INFO
>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>> RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
>> at akka://flink/user/dispatcher .
>> 2018-08-29 13:19:26,335 INFO
>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>> Starting ZooKeeperLeaderElectionService
>> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
>> 2018-08-29 13:19:26,336 INFO
>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>> Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>> 2018-08-29 13:19:26,338 INFO
>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>> Starting ZooKeeperLeaderElectionService
>> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
>> 2018-08-29 13:19:26,339 INFO
>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>> 2018-08-29 13:23:21,513 INFO
>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>> Recovered SubmittedJobGraph(836b29f7c66bbeb6ed8bae41cb9b316c, null).
>>
>> Thanks,
>> Encho
>>
>> On Wed, Aug 29, 2018 at 3:59 PM Till Rohrmann <tr...@apache.org>
>> wrote:
>>
>>> Hi Encho,
>>>
>>> thanks for sending the first part of the logs. What I would actually be
>>> interested in are the complete logs because somewhere in the jobmanager-2
>>> logs there must be a log statement saying that the respective dispatcher
>>> gained leadership. I would like to see why this happens but for this to
>>> debug the complete logs are necessary. It would be awesome if you could
>>> send them to me. Thanks a lot!
>>>
>>> Cheers,
>>> Till
>>>
>>> On Wed, Aug 29, 2018 at 2:00 PM Encho Mishinev <en...@gmail.com>
>>> wrote:
>>>
>>>> Hi Till,
>>>>
>>>> I will use the approach with a k8s deployment and HA mode with a single
>>>> job manager. Nonetheless, here are the logs I just produced by repeating
>>>> the aforementioned experiment, hope they help in debugging:
>>>>
>>>> *- Starting Jobmanager-1:*
>>>>
>>>> Starting Job Manager
>>>> sed: cannot rename /opt/flink/conf/sedR98XPn: Device or resource busy
>>>> config file:
>>>> jobmanager.rpc.address: flink-jobmanager-1
>>>> jobmanager.rpc.port: 6123
>>>> jobmanager.heap.size: 8192
>>>> taskmanager.heap.size: 8192
>>>> taskmanager.numberOfTaskSlots: 4
>>>> high-availability: zookeeper
>>>> high-availability.storageDir:
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>>>> high-availability.zookeeper.quorum: zk-cs:2181
>>>> high-availability.zookeeper.path.root: /flink
>>>> high-availability.jobmanager.port: 50010
>>>> state.backend: filesystem
>>>> state.checkpoints.dir:
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>>>> state.savepoints.dir:
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>>>> state.backend.incremental: false
>>>> fs.default-scheme:
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>>>> rest.port: 8081
>>>> web.upload.dir: /opt/flink/upload
>>>> query.server.port: 6125
>>>> taskmanager.numberOfTaskSlots: 4
>>>> classloader.parent-first-patterns.additional: org.apache.xerces.
>>>> blob.storage.directory: /opt/flink/blob-server
>>>> blob.server.port: 6124
>>>> blob.server.port: 6124
>>>> query.server.port: 6125
>>>> Starting standalonesession as a console application on host
>>>> flink-jobmanager-1-f76fd4df8-ftwt9.
>>>> 2018-08-29 11:41:48,806 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>> --------------------------------------------------------------------------------
>>>> 2018-08-29 11:41:48,807 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
>>>> StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
>>>> Date:16.08.2018 @ 06:39:50 GMT)
>>>> 2018-08-29 11:41:48,807 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
>>>> user: flink
>>>> 2018-08-29 11:41:49,134 WARN  org.apache.hadoop.util.NativeCodeLoader
>>>>                      - Unable to load native-hadoop library for your
>>>> platform... using builtin-java classes where applicable
>>>> 2018-08-29 11:41:49,210 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
>>>> Hadoop/Kerberos user: flink
>>>> 2018-08-29 11:41:49,210 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
>>>> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
>>>> 2018-08-29 11:41:49,210 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
>>>> heap size: 6702 MiBytes
>>>> 2018-08-29 11:41:49,210 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
>>>> /docker-java-home/jre
>>>> 2018-08-29 11:41:49,213 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
>>>> version: 2.7.5
>>>> 2018-08-29 11:41:49,213 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
>>>> Options:
>>>> 2018-08-29 11:41:49,213 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>>  -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
>>>> 2018-08-29 11:41:49,213 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>>  -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
>>>> 2018-08-29 11:41:49,213 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
>>>> Arguments:
>>>> 2018-08-29 11:41:49,213 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>>  --configDir
>>>> 2018-08-29 11:41:49,213 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>>  /opt/flink/conf
>>>> 2018-08-29 11:41:49,213 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>>  --executionMode
>>>> 2018-08-29 11:41:49,213 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>>>> 2018-08-29 11:41:49,214 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
>>>> 2018-08-29 11:41:49,214 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>>>> 2018-08-29 11:41:49,214 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
>>>> /opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>>>> 2018-08-29 11:41:49,214 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>> --------------------------------------------------------------------------------
>>>> 2018-08-29 11:41:49,215 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
>>>> UNIX signal handlers for [TERM, HUP, INT]
>>>> 2018-08-29 11:41:49,221 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: jobmanager.rpc.address, flink-jobmanager-1
>>>> 2018-08-29 11:41:49,221 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: jobmanager.rpc.port, 6123
>>>> 2018-08-29 11:41:49,221 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: jobmanager.heap.size, 8192
>>>> 2018-08-29 11:41:49,221 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: taskmanager.heap.size, 8192
>>>> 2018-08-29 11:41:49,221 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: taskmanager.numberOfTaskSlots, 4
>>>> 2018-08-29 11:41:49,222 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: high-availability, zookeeper
>>>> 2018-08-29 11:41:49,222 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: high-availability.storageDir,
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>>>> 2018-08-29 11:41:49,222 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: high-availability.zookeeper.quorum, zk-cs:2181
>>>> 2018-08-29 11:41:49,222 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: high-availability.zookeeper.path.root, /flink
>>>> 2018-08-29 11:41:49,223 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: high-availability.jobmanager.port, 50010
>>>> 2018-08-29 11:41:49,223 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: state.backend, filesystem
>>>> 2018-08-29 11:41:49,223 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: state.checkpoints.dir,
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>>>> 2018-08-29 11:41:49,223 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: state.savepoints.dir,
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>>>> 2018-08-29 11:41:49,223 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: state.backend.incremental, false
>>>> 2018-08-29 11:41:49,224 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: fs.default-scheme,
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>>>> 2018-08-29 11:41:49,224 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: rest.port, 8081
>>>> 2018-08-29 11:41:49,224 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: web.upload.dir, /opt/flink/upload
>>>> 2018-08-29 11:41:49,224 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: query.server.port, 6125
>>>> 2018-08-29 11:41:49,225 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: taskmanager.numberOfTaskSlots, 4
>>>> 2018-08-29 11:41:49,225 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: classloader.parent-first-patterns.additional,
>>>> org.apache.xerces.
>>>> 2018-08-29 11:41:49,225 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: blob.storage.directory, /opt/flink/blob-server
>>>> 2018-08-29 11:41:49,225 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: blob.server.port, 6124
>>>> 2018-08-29 11:41:49,225 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: blob.server.port, 6124
>>>> 2018-08-29 11:41:49,225 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: query.server.port, 6125
>>>> 2018-08-29 11:41:49,239 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
>>>> StandaloneSessionClusterEntrypoint.
>>>> 2018-08-29 11:41:49,239 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>>>> default filesystem.
>>>> 2018-08-29 11:41:49,250 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>>>> security context.
>>>> 2018-08-29 11:41:49,282 INFO
>>>> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
>>>> set to flink (auth:SIMPLE)
>>>> 2018-08-29 11:41:49,298 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>> Initializing cluster services.
>>>> 2018-08-29 11:41:49,309 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
>>>> start actor system at flink-jobmanager-1:50010
>>>> 2018-08-29 11:41:49,768 INFO  akka.event.slf4j.Slf4jLogger
>>>>                     - Slf4jLogger started
>>>> 2018-08-29 11:41:49,823 INFO  akka.remote.Remoting
>>>>                     - Starting remoting
>>>> 2018-08-29 11:41:49,974 INFO  akka.remote.Remoting
>>>>                     - Remoting started; listening on addresses
>>>> :[akka.tcp://flink@flink-jobmanager-1:50010]
>>>> 2018-08-29 11:41:49,981 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
>>>> system started at akka.tcp://flink@flink-jobmanager-1:50010
>>>> 2018-08-29 11:41:50,444 INFO
>>>> org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
>>>> highly available BLOB storage directory at
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
>>>> 2018-08-29 11:41:50,509 INFO
>>>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing
>>>> default ACL for ZK connections
>>>> 2018-08-29 11:41:50,509 INFO
>>>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Using
>>>> '/flink/default' as Zookeeper namespace.
>>>> 2018-08-29 11:41:50,568 INFO
>>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
>>>> - Starting
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
>>>> built on 03/23/2017 10:13 GMT
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:host.name=flink-jobmanager-1-f76fd4df8-ftwt9
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.version=1.8.0_181
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.vendor=Oracle Corporation
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.io.tmpdir=/tmp
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.compiler=<NA>
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:os.name=Linux
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:os.arch=amd64
>>>> 2018-08-29 11:41:50,577 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:os.version=4.4.0-1027-gke
>>>> 2018-08-29 11:41:50,578 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:user.name=flink
>>>> 2018-08-29 11:41:50,578 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:user.home=/opt/flink
>>>> 2018-08-29 11:41:50,578 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:user.dir=/opt/flink
>>>> 2018-08-29 11:41:50,578 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
>>>> Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
>>>> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
>>>> 2018-08-29 11:41:50,605 INFO  org.apache.flink.runtime.blob.BlobServer
>>>>                     - Created BLOB server storage directory
>>>> /opt/flink/blob-server/blobStore-d408cea8-2ed0-461a-a30a-a62b70fd332a
>>>> 2018-08-29 11:41:50,605 WARN
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
>>>> configuration failed: javax.security.auth.login.LoginException: No JAAS
>>>> configuration section named 'Client' was found in specified JAAS
>>>> configuration file: '/tmp/jaas-5372401662150571998.conf'. Will continue
>>>> connection to Zookeeper server without SASL authentication, if Zookeeper
>>>> server allows it.
>>>> 2018-08-29 11:41:50,607 INFO  org.apache.flink.runtime.blob.BlobServer
>>>>                     - Started BLOB server at 0.0.0.0:6124 - max
>>>> concurrent requests: 50 - max backlog: 1000
>>>> 2018-08-29 11:41:50,607 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>>>> Opening socket connection to server zk-cs.default.svc.cluster.local/
>>>> 10.27.248.104:2181
>>>> 2018-08-29 11:41:50,608 ERROR
>>>> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
>>>> Authentication failed
>>>> 2018-08-29 11:41:50,609 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
>>>> connection established to zk-cs.default.svc.cluster.local/
>>>> 10.27.248.104:2181, initiating session
>>>> 2018-08-29 11:41:50,618 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>>>> Session establishment complete on server zk-cs.default.svc.cluster.local/
>>>> 10.27.248.104:2181, sessionid = 0x26584fd55690005, negotiated timeout
>>>> = 40000
>>>> 2018-08-29 11:41:50,619 INFO
>>>> org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
>>>> - State change: CONNECTED
>>>> 2018-08-29 11:41:50,627 INFO
>>>> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
>>>> reporter configured, no metrics will be exposed/reported.
>>>> 2018-08-29 11:41:50,633 INFO
>>>> org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
>>>> Initializing FileArchivedExecutionGraphStore: Storage directory
>>>> /tmp/executionGraphStore-c5df0b39-86f3-4fba-bdda-aacca4f86086, expiration
>>>> time 3600000, maximum cache size 52428800 bytes.
>>>> 2018-08-29 11:41:50,659 INFO
>>>> org.apache.flink.runtime.blob.TransientBlobCache              - Created
>>>> BLOB cache storage directory
>>>> /opt/flink/blob-server/blobStore-c12d55af-3c2d-4fc2-8ee8-6de642522184
>>>> 2018-08-29 11:41:50,674 WARN
>>>> org.apache.flink.configuration.Configuration                  - Config uses
>>>> deprecated configuration key 'jobmanager.rpc.address' instead of proper key
>>>> 'rest.address'
>>>> 2018-08-29 11:41:50,675 WARN
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
>>>> directory /opt/flink/upload/flink-web-upload does not exist, or has been
>>>> deleted externally. Previously uploaded files are no longer available.
>>>> 2018-08-29 11:41:50,676 INFO
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
>>>> directory /opt/flink/upload/flink-web-upload for file uploads.
>>>> 2018-08-29 11:41:50,679 INFO
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
>>>> rest endpoint.
>>>> 2018-08-29 11:41:50,995 WARN
>>>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
>>>> environment variable 'log.file' is not set.
>>>> 2018-08-29 11:41:50,995 WARN
>>>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
>>>> log files are unavailable in the web dashboard. Log file location not found
>>>> in environment variable 'log.file' or configuration key 'Key:
>>>> 'web.log.path' , default: null (deprecated keys:
>>>> [jobmanager.web.log.path])'.
>>>> 2018-08-29 11:41:51,071 INFO
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
>>>> endpoint listening at flink-jobmanager-1:8081
>>>> 2018-08-29 11:41:51,071 INFO
>>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>>> Starting ZooKeeperLeaderElectionService
>>>> ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
>>>> 2018-08-29 11:41:51,091 WARN
>>>> org.apache.flink.shaded.curator.org.apache.curator.utils.ZKPaths  - The
>>>> version of ZooKeeper being used doesn't support Container nodes.
>>>> CreateMode.PERSISTENT will be used instead.
>>>> 2018-08-29 11:41:51,101 INFO
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
>>>> frontend listening at http://flink-jobmanager-1:8081.
>>>> 2018-08-29 11:41:51,114 INFO
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>>>> RPC endpoint for
>>>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
>>>> akka://flink/user/resourcemanager .
>>>> 2018-08-29 11:41:51,141 INFO
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    -
>>>> http://flink-jobmanager-1:8081 was granted leadership with
>>>> leaderSessionID=bb0d4dfd-c2c4-480b-bc86-62e231a606dd
>>>> 2018-08-29 11:41:51,214 INFO
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>>>> RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
>>>> at akka://flink/user/dispatcher .
>>>> 2018-08-29 11:41:51,230 INFO
>>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>>> Starting ZooKeeperLeaderElectionService
>>>> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
>>>> 2018-08-29 11:41:51,232 INFO
>>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>>> Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>>>> 2018-08-29 11:41:51,234 INFO
>>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>>> Starting ZooKeeperLeaderElectionService
>>>> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
>>>> 2018-08-29 11:41:51,235 INFO
>>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>>> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>>> 2018-08-29 11:41:51,253 INFO
>>>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  -
>>>> ResourceManager akka.tcp://flink@flink-jobmanager-1:50010/user/resourcemanager
>>>> was granted leadership with fencing token ba47ed8daa8ff16bea6fc355c13f4d49
>>>> 2018-08-29 11:41:51,254 INFO
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
>>>> Starting the SlotManager.
>>>> 2018-08-29 11:41:51,263 INFO
>>>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Dispatcher
>>>> akka.tcp://flink@flink-jobmanager-1:50010/user/dispatcher was granted
>>>> leadership with fencing token 703301bf-85e7-4464-990f-ad39128a7b4d
>>>> 2018-08-29 11:41:51,263 INFO
>>>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Recovering
>>>> all persisted jobs.
>>>> 2018-08-29 11:41:51,468 INFO
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
>>>> Registering TaskManager c8a3201d58d87dbbe16f8eb352b5c5b6 under
>>>> 1c5bf0bc3848bd384b6f032ff7213754 at the SlotManager.
>>>> 2018-08-29 11:41:51,471 INFO
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
>>>> Registering TaskManager 104d18b72fed054620e58e120a1ea083 under
>>>> e9d3e8ad3b477dd2e58bcb88a2c0d061 at the SlotManager.
>>>>
>>>> *Starting Jobmanager-2:*
>>>>
>>>> Starting Job Manager
>>>> sed: cannot rename /opt/flink/conf/sedH2ZiSu: Device or resource busy
>>>> config file:
>>>> jobmanager.rpc.address: flink-jobmanager-2
>>>> jobmanager.rpc.port: 6123
>>>> jobmanager.heap.size: 8192
>>>> taskmanager.heap.size: 8192
>>>> taskmanager.numberOfTaskSlots: 4
>>>> high-availability: zookeeper
>>>> high-availability.storageDir:
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>>>> high-availability.zookeeper.quorum: zk-cs:2181
>>>> high-availability.zookeeper.path.root: /flink
>>>> high-availability.jobmanager.port: 50010
>>>> state.backend: filesystem
>>>> state.checkpoints.dir:
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>>>> state.savepoints.dir:
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>>>> state.backend.incremental: false
>>>> fs.default-scheme:
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>>>> rest.port: 8081
>>>> web.upload.dir: /opt/flink/upload
>>>> query.server.port: 6125
>>>> taskmanager.numberOfTaskSlots: 4
>>>> classloader.parent-first-patterns.additional: org.apache.xerces.
>>>> blob.storage.directory: /opt/flink/blob-server
>>>> blob.server.port: 6124
>>>> blob.server.port: 6124
>>>> query.server.port: 6125
>>>> Starting standalonesession as a console application on host
>>>> flink-jobmanager-2-7844b78c9-kmvw9.
>>>> 2018-08-29 11:41:51,688 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>> --------------------------------------------------------------------------------
>>>> 2018-08-29 11:41:51,690 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
>>>> StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
>>>> Date:16.08.2018 @ 06:39:50 GMT)
>>>> 2018-08-29 11:41:51,690 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
>>>> user: flink
>>>> 2018-08-29 11:41:52,018 WARN  org.apache.hadoop.util.NativeCodeLoader
>>>>                      - Unable to load native-hadoop library for your
>>>> platform... using builtin-java classes where applicable
>>>> 2018-08-29 11:41:52,088 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
>>>> Hadoop/Kerberos user: flink
>>>> 2018-08-29 11:41:52,088 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
>>>> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
>>>> 2018-08-29 11:41:52,088 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
>>>> heap size: 6702 MiBytes
>>>> 2018-08-29 11:41:52,088 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
>>>> /docker-java-home/jre
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
>>>> version: 2.7.5
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
>>>> Options:
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>>  -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>>  -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
>>>> Arguments:
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>>  --configDir
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>>  /opt/flink/conf
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>>  --executionMode
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
>>>> /opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>>>> 2018-08-29 11:41:52,091 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>> --------------------------------------------------------------------------------
>>>> 2018-08-29 11:41:52,092 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
>>>> UNIX signal handlers for [TERM, HUP, INT]
>>>> 2018-08-29 11:41:52,103 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: jobmanager.rpc.address, flink-jobmanager-2
>>>> 2018-08-29 11:41:52,103 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: jobmanager.rpc.port, 6123
>>>> 2018-08-29 11:41:52,103 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: jobmanager.heap.size, 8192
>>>> 2018-08-29 11:41:52,104 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: taskmanager.heap.size, 8192
>>>> 2018-08-29 11:41:52,104 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: taskmanager.numberOfTaskSlots, 4
>>>> 2018-08-29 11:41:52,104 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: high-availability, zookeeper
>>>> 2018-08-29 11:41:52,104 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: high-availability.storageDir,
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>>>> 2018-08-29 11:41:52,104 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: high-availability.zookeeper.quorum, zk-cs:2181
>>>> 2018-08-29 11:41:52,104 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: high-availability.zookeeper.path.root, /flink
>>>> 2018-08-29 11:41:52,105 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: high-availability.jobmanager.port, 50010
>>>> 2018-08-29 11:41:52,105 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: state.backend, filesystem
>>>> 2018-08-29 11:41:52,105 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: state.checkpoints.dir,
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>>>> 2018-08-29 11:41:52,105 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: state.savepoints.dir,
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>>>> 2018-08-29 11:41:52,105 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: state.backend.incremental, false
>>>> 2018-08-29 11:41:52,106 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: fs.default-scheme,
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>>>> 2018-08-29 11:41:52,106 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: rest.port, 8081
>>>> 2018-08-29 11:41:52,106 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: web.upload.dir, /opt/flink/upload
>>>> 2018-08-29 11:41:52,106 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: query.server.port, 6125
>>>> 2018-08-29 11:41:52,106 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: taskmanager.numberOfTaskSlots, 4
>>>> 2018-08-29 11:41:52,107 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: classloader.parent-first-patterns.additional,
>>>> org.apache.xerces.
>>>> 2018-08-29 11:41:52,107 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: blob.storage.directory, /opt/flink/blob-server
>>>> 2018-08-29 11:41:52,107 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: blob.server.port, 6124
>>>> 2018-08-29 11:41:52,107 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: blob.server.port, 6124
>>>> 2018-08-29 11:41:52,107 INFO
>>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>>> configuration property: query.server.port, 6125
>>>> 2018-08-29 11:41:52,122 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
>>>> StandaloneSessionClusterEntrypoint.
>>>> 2018-08-29 11:41:52,123 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>>>> default filesystem.
>>>> 2018-08-29 11:41:52,133 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>>>> security context.
>>>> 2018-08-29 11:41:52,173 INFO
>>>> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
>>>> set to flink (auth:SIMPLE)
>>>> 2018-08-29 11:41:52,188 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>> Initializing cluster services.
>>>> 2018-08-29 11:41:52,198 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
>>>> start actor system at flink-jobmanager-2:50010
>>>> 2018-08-29 11:41:52,753 INFO  akka.event.slf4j.Slf4jLogger
>>>>                     - Slf4jLogger started
>>>> 2018-08-29 11:41:52,822 INFO  akka.remote.Remoting
>>>>                     - Starting remoting
>>>> 2018-08-29 11:41:53,038 INFO  akka.remote.Remoting
>>>>                     - Remoting started; listening on addresses
>>>> :[akka.tcp://flink@flink-jobmanager-2:50010]
>>>> 2018-08-29 11:41:53,046 INFO
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
>>>> system started at akka.tcp://flink@flink-jobmanager-2:50010
>>>> 2018-08-29 11:41:53,500 INFO
>>>> org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
>>>> highly available BLOB storage directory at
>>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
>>>> 2018-08-29 11:41:53,558 INFO
>>>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing
>>>> default ACL for ZK connections
>>>> 2018-08-29 11:41:53,559 INFO
>>>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Using
>>>> '/flink/default' as Zookeeper namespace.
>>>> 2018-08-29 11:41:53,616 INFO
>>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
>>>> - Starting
>>>> 2018-08-29 11:41:53,624 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
>>>> built on 03/23/2017 10:13 GMT
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:host.name=flink-jobmanager-2-7844b78c9-kmvw9
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.version=1.8.0_181
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.vendor=Oracle Corporation
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.io.tmpdir=/tmp
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:java.compiler=<NA>
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:os.name=Linux
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:os.arch=amd64
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:os.version=4.4.0-1027-gke
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:user.name=flink
>>>> 2018-08-29 11:41:53,625 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:user.home=/opt/flink
>>>> 2018-08-29 11:41:53,626 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>> environment:user.dir=/opt/flink
>>>> 2018-08-29 11:41:53,626 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
>>>> Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
>>>> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
>>>> 2018-08-29 11:41:53,644 WARN
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
>>>> configuration failed: javax.security.auth.login.LoginException: No JAAS
>>>> configuration section named 'Client' was found in specified JAAS
>>>> configuration file: '/tmp/jaas-8238466329925822361.conf'. Will continue
>>>> connection to Zookeeper server without SASL authentication, if Zookeeper
>>>> server allows it.
>>>> 2018-08-29 11:41:53,646 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>>>> Opening socket connection to server zk-cs.default.svc.cluster.local/
>>>> 10.27.248.104:2181
>>>> 2018-08-29 11:41:53,646 INFO  org.apache.flink.runtime.blob.BlobServer
>>>>                     - Created BLOB server storage directory
>>>> /opt/flink/blob-server/blobStore-61cdb645-5d0c-47fd-bcf6-84ad16fadade
>>>> 2018-08-29 11:41:53,646 ERROR
>>>> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
>>>> Authentication failed
>>>> 2018-08-29 11:41:53,647 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
>>>> connection established to zk-cs.default.svc.cluster.local/
>>>> 10.27.248.104:2181, initiating session
>>>> 2018-08-29 11:41:53,649 INFO  org.apache.flink.runtime.blob.BlobServer
>>>>                     - Started BLOB server at 0.0.0.0:6124 - max
>>>> concurrent requests: 50 - max backlog: 1000
>>>> 2018-08-29 11:41:53,655 INFO
>>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>>>> Session establishment complete on server zk-cs.default.svc.cluster.local/
>>>> 10.27.248.104:2181, sessionid = 0x26584fd55690006, negotiated timeout
>>>> = 40000
>>>> 2018-08-29 11:41:53,656 INFO
>>>> org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
>>>> - State change: CONNECTED
>>>> 2018-08-29 11:41:53,667 INFO
>>>> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
>>>> reporter configured, no metrics will be exposed/reported.
>>>> 2018-08-29 11:41:53,673 INFO
>>>> org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
>>>> Initializing FileArchivedExecutionGraphStore: Storage directory
>>>> /tmp/executionGraphStore-8b236c14-79ee-4a84-b23f-437408c4661a, expiration
>>>> time 3600000, maximum cache size 52428800 bytes.
>>>> 2018-08-29 11:41:53,699 INFO
>>>> org.apache.flink.runtime.blob.TransientBlobCache              - Created
>>>> BLOB cache storage directory
>>>> /opt/flink/blob-server/blobStore-80c519df-cc6f-4e9c-9cd5-da4077c826f0
>>>> 2018-08-29 11:41:53,717 WARN
>>>> org.apache.flink.configuration.Configuration                  - Config uses
>>>> deprecated configuration key 'jobmanager.rpc.address' instead of proper key
>>>> 'rest.address'
>>>> 2018-08-29 11:41:53,718 WARN
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
>>>> directory /opt/flink/upload/flink-web-upload does not exist, or has been
>>>> deleted externally. Previously uploaded files are no longer available.
>>>> 2018-08-29 11:41:53,719 INFO
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
>>>> directory /opt/flink/upload/flink-web-upload for file uploads.
>>>> 2018-08-29 11:41:53,722 INFO
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
>>>> rest endpoint.
>>>> 2018-08-29 11:41:54,084 WARN
>>>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
>>>> environment variable 'log.file' is not set.
>>>> 2018-08-29 11:41:54,084 WARN
>>>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
>>>> log files are unavailable in the web dashboard. Log file location not found
>>>> in environment variable 'log.file' or configuration key 'Key:
>>>> 'web.log.path' , default: null (deprecated keys:
>>>> [jobmanager.web.log.path])'.
>>>> 2018-08-29 11:41:54,160 INFO
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
>>>> endpoint listening at flink-jobmanager-2:8081
>>>> 2018-08-29 11:41:54,160 INFO
>>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>>> Starting ZooKeeperLeaderElectionService
>>>> ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
>>>> 2018-08-29 11:41:54,180 INFO
>>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
>>>> frontend listening at http://flink-jobmanager-2:8081.
>>>> 2018-08-29 11:41:54,192 INFO
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>>>> RPC endpoint for
>>>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
>>>> akka://flink/user/resourcemanager .
>>>> 2018-08-29 11:41:54,273 INFO
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>>>> RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
>>>> at akka://flink/user/dispatcher .
>>>> 2018-08-29 11:41:54,286 INFO
>>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>>> Starting ZooKeeperLeaderElectionService
>>>> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
>>>> 2018-08-29 11:41:54,287 INFO
>>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>>> Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>>>> 2018-08-29 11:41:54,289 INFO
>>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>>> Starting ZooKeeperLeaderElectionService
>>>> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
>>>> 2018-08-29 11:41:54,289 INFO
>>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>>> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>>>
>>>> *Upon submitting a batch job on Jobmanager-1, we immediately get this
>>>> log on Jobmanager-2*
>>>> 2018-08-29 11:47:06,249 INFO
>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>> Recovered SubmittedJobGraph(d69b67e4d28a2d244b06d3f6d661bca1, null).
>>>>
>>>> *Meanwhile Jobmanager-1 gets:*
>>>> *-FlinkBatchPipelineTranslator pipeline logs- (we use Apache Beam)*
>>>>
>>>> 2018-08-29 11:47:06,006 INFO
>>>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Submitting
>>>> job d69b67e4d28a2d244b06d3f6d661bca1
>>>> (sicassandrawriterbeam-flink-0829114703-7d95fabd).
>>>> 2018-08-29 11:47:06,090 INFO
>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>> Added SubmittedJobGraph(d69b67e4d28a2d244b06d3f6d661bca1, null) to
>>>> ZooKeeper.
>>>>
>>>> *-loads of job execution info-*
>>>>
>>>> 2018-08-29 11:49:20,272 INFO
>>>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Job
>>>> d69b67e4d28a2d244b06d3f6d661bca1 reached globally terminal state FINISHED.
>>>> 2018-08-29 11:49:20,286 INFO
>>>> org.apache.flink.runtime.jobmaster.JobMaster                  - Stopping
>>>> the JobMaster for job
>>>> sicassandrawriterbeam-flink-0829114703-7d95fabd(d69b67e4d28a2d244b06d3f6d661bca1).
>>>> 2018-08-29 11:49:20,290 INFO
>>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>>> Stopping ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>>>> 2018-08-29 11:49:20,292 INFO
>>>> org.apache.flink.runtime.jobmaster.JobMaster                  - Close
>>>> ResourceManager connection 827b94881bf7c94d8516907e04e3a564: JobManager is
>>>> shutting down..
>>>> 2018-08-29 11:49:20,292 INFO
>>>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Suspending
>>>> SlotPool.
>>>> 2018-08-29 11:49:20,293 INFO
>>>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Stopping
>>>> SlotPool.
>>>> 2018-08-29 11:49:20,293 INFO
>>>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  -
>>>> Disconnect job manager a3dab0a0883c5f0f37943358d9104d79
>>>> @akka.tcp://flink@flink-jobmanager-1:50010/user/jobmanager_0 for job
>>>> d69b67e4d28a2d244b06d3f6d661bca1 from the resource manager.
>>>> 2018-08-29 11:49:20,293 INFO
>>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>>> Stopping ZooKeeperLeaderElectionService
>>>> ZooKeeperLeaderElectionService{leaderPath='/leader/d69b67e4d28a2d244b06d3f6d661bca1/job_manager_lock'}.
>>>> 2018-08-29 11:49:20,304 INFO
>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>> Removed job graph d69b67e4d28a2d244b06d3f6d661bca1 from ZooKeeper.
>>>>
>>>>
>>>> -------------------
>>>>
>>>> The result is:
>>>> HDFS has only a jobgraph and an empty default folder - everything else
>>>> is cleared
>>>> ZooKeeper has the jobgraph that Jobmanager-1 claims to have removed in
>>>> the last log still there.
>>>>
>>>> On Wed, Aug 29, 2018 at 12:14 PM Till Rohrmann <tr...@apache.org>
>>>> wrote:
>>>>
>>>>> Hi Encho,
>>>>>
>>>>> it sounds strange that the standby JobManager tries to recover a
>>>>> submitted job graph. This should only happen if it has been granted
>>>>> leadership. Thus, it seems as if the standby JobManager thinks that it is
>>>>> also the leader. Could you maybe share the logs of the two
>>>>> JobManagers/ClusterEntrypoints with us?
>>>>>
>>>>> Running only a single JobManager/ClusterEntrypoint in HA mode via a
>>>>> Kubernetes Deployment should do the trick and there is nothing wrong with
>>>>> it.
>>>>>
>>>>> Cheers,
>>>>> Till
>>>>>
>>>>> On Wed, Aug 29, 2018 at 11:05 AM Encho Mishinev <
>>>>> encho.mishinev@gmail.com> wrote:
>>>>>
>>>>>> Hello,
>>>>>>
>>>>>> Since two job managers don't seem to be working for me I was thinking
>>>>>> of just using a single job manager in Kubernetes in HA mode with a
>>>>>> deployment ensuring its restart whenever it fails. Is this approach viable?
>>>>>> The High-Availability page mentions that you use only one job manager in an
>>>>>> YARN cluster but does not specify such option for Kubernetes. Is there
>>>>>> anything that can go wrong with this approach?
>>>>>>
>>>>>> Thanks
>>>>>>
>>>>>> On Wed, Aug 29, 2018 at 11:10 AM Encho Mishinev <
>>>>>> encho.mishinev@gmail.com> wrote:
>>>>>>
>>>>>>> Hi,
>>>>>>>
>>>>>>> Unfortunately the thing I described does indeed happen every time.
>>>>>>> As mentioned in the first email, I am running on Kubernetes so certain
>>>>>>> things could be different compared to just a standalone cluster.
>>>>>>>
>>>>>>> Any ideas for workarounds are welcome, as this problem basically
>>>>>>> prevents me from using HA.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Encho
>>>>>>>
>>>>>>> On Wed, Aug 29, 2018 at 5:15 AM vino yang <ya...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Encho,
>>>>>>>>
>>>>>>>> From your description, I feel that there are extra bugs.
>>>>>>>>
>>>>>>>> About your description:
>>>>>>>>
>>>>>>>> *- Start both job managers*
>>>>>>>> *- Start a batch job in JobManager 1 and let it finish*
>>>>>>>> *The jobgraphs in both Zookeeper and HDFS remained.*
>>>>>>>>
>>>>>>>> Is it necessarily happening every time?
>>>>>>>>
>>>>>>>> In the Standalone cluster, the problems we encountered were
>>>>>>>> sporadic.
>>>>>>>>
>>>>>>>> Thanks, vino.
>>>>>>>>
>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午8:07写道:
>>>>>>>>
>>>>>>>>> Hello Till,
>>>>>>>>>
>>>>>>>>> I spend a few more hours testing and looking at the logs and it
>>>>>>>>> seems like there's a more general problem here. While the two job managers
>>>>>>>>> are active neither of them can properly delete jobgraphs. The above problem
>>>>>>>>> I described comes from the fact that Kubernetes gets JobManager 1 quickly
>>>>>>>>> after I manually kill it, so when I stop the job on JobManager 2 both are
>>>>>>>>> alive.
>>>>>>>>>
>>>>>>>>> I did a very simple test:
>>>>>>>>>
>>>>>>>>> - Start both job managers
>>>>>>>>> - Start a batch job in JobManager 1 and let it finish
>>>>>>>>> The jobgraphs in both Zookeeper and HDFS remained.
>>>>>>>>>
>>>>>>>>> On the other hand if we do:
>>>>>>>>>
>>>>>>>>> - Start only JobManager 1 (again in HA mode)
>>>>>>>>> - Start a batch job and let it finish
>>>>>>>>> The jobgraphs in both Zookeeper and HDFS are deleted fine.
>>>>>>>>>
>>>>>>>>> It seems like the standby manager still leaves some kind of lock
>>>>>>>>> on the jobgraphs. Do you think that's possible? Have you seen a similar
>>>>>>>>> problem?
>>>>>>>>> The only logs that appear on the standby manager while waiting are
>>>>>>>>> of the type:
>>>>>>>>>
>>>>>>>>> 2018-08-28 11:54:10,789 INFO
>>>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>>>> Recovered SubmittedJobGraph(9e0a109b57511930c95d3b54574a66e3, null).
>>>>>>>>>
>>>>>>>>> Note that this log appears on the standby jobmanager immediately
>>>>>>>>> when a new job is submitted to the active jobmanager.
>>>>>>>>> Also note that the blobs and checkpoints are cleared fine. The
>>>>>>>>> problem is only for jobgraphs both in ZooKeeper and HDFS.
>>>>>>>>>
>>>>>>>>> Trying to access the UI of the standby manager redirects to the
>>>>>>>>> active one, so it is not a problem of them not knowing who the leader is.
>>>>>>>>> Do you have any ideas?
>>>>>>>>>
>>>>>>>>> Thanks a lot,
>>>>>>>>> Encho
>>>>>>>>>
>>>>>>>>> On Tue, Aug 28, 2018 at 10:27 AM Till Rohrmann <
>>>>>>>>> trohrmann@apache.org> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Encho,
>>>>>>>>>>
>>>>>>>>>> thanks a lot for reporting this issue. The problem arises
>>>>>>>>>> whenever the old leader maintains the connection to ZooKeeper. If this is
>>>>>>>>>> the case, then ephemeral nodes which we create to protect against faulty
>>>>>>>>>> delete operations are not removed and consequently the new leader is not
>>>>>>>>>> able to delete the persisted job graph. So one thing to check is whether
>>>>>>>>>> the old JM still has an open connection to ZooKeeper. The next thing to
>>>>>>>>>> check is the session timeout of your ZooKeeper cluster. If you stop the job
>>>>>>>>>> within the session timeout, then it is also not guaranteed that ZooKeeper
>>>>>>>>>> has detected that the ephemeral nodes of the old JM must be deleted. In
>>>>>>>>>> order to understand this better it would be helpful if you could tell us
>>>>>>>>>> the timing of the different actions.
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Till
>>>>>>>>>>
>>>>>>>>>> On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Encho,
>>>>>>>>>>>
>>>>>>>>>>> A temporary solution can be used to determine if it has been
>>>>>>>>>>> cleaned up by monitoring the specific JobID under Zookeeper's "/jobgraph".
>>>>>>>>>>> Another solution, modify the source code, rudely modify the
>>>>>>>>>>> cleanup mode to the synchronous form, but the flink operation Zookeeper's
>>>>>>>>>>> path needs to obtain the corresponding lock, so it is dangerous to do so,
>>>>>>>>>>> and it is not recommended.
>>>>>>>>>>> I think maybe this problem can be solved in the next version. It
>>>>>>>>>>> depends on Till.
>>>>>>>>>>>
>>>>>>>>>>> Thanks, vino.
>>>>>>>>>>>
>>>>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二
>>>>>>>>>>> 下午1:17写道:
>>>>>>>>>>>
>>>>>>>>>>>> Thank you very much for the info! Will keep track of the
>>>>>>>>>>>> progress.
>>>>>>>>>>>>
>>>>>>>>>>>> In the meantime is there any viable workaround? It seems like
>>>>>>>>>>>> HA doesn't really work due to this bug.
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <
>>>>>>>>>>>> yanghua1127@gmail.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> About some implementation mechanisms.
>>>>>>>>>>>>> Flink uses Zookeeper to store JobGraph (Job's description
>>>>>>>>>>>>> information and metadata) as a basis for Job recovery.
>>>>>>>>>>>>> However, previous implementations may cause this information
>>>>>>>>>>>>> to not be properly cleaned up because it is asynchronously deleted by a
>>>>>>>>>>>>> background thread.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks, vino.
>>>>>>>>>>>>>
>>>>>>>>>>>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Encho,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> This is a problem already known to the Flink community, you
>>>>>>>>>>>>>> can track its progress through FLINK-10011[1], and currently Till is fixing
>>>>>>>>>>>>>> this issue.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks, vino.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一
>>>>>>>>>>>>>> 下午10:13写道:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I am running Flink 1.5.3 with two job managers and two task
>>>>>>>>>>>>>>> managers in Kubernetes along with HDFS and Zookeeper in high-availability
>>>>>>>>>>>>>>> mode.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> My problem occurs after the following actions:
>>>>>>>>>>>>>>> - Upload a .jar file to jobmanager-1
>>>>>>>>>>>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>>>>>>>>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>>>>>>>>>>>> - Kill pod of jobmanager-1
>>>>>>>>>>>>>>> After a short delay, jobmanager-2 takes leadership and
>>>>>>>>>>>>>>> correctly restores the job and continues it
>>>>>>>>>>>>>>> - Stop job from jobmanager-2
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> At this point all seems well, but the problem is that
>>>>>>>>>>>>>>> jobmanager-2 does not clean up anything that was left from jobmanager-1.
>>>>>>>>>>>>>>> This means that both in HDFS and in Zookeeper remain job graphs, which
>>>>>>>>>>>>>>> later on obstruct any work of both managers as after any reset they
>>>>>>>>>>>>>>> unsuccessfully try to restore a non-existent job and fail over and over
>>>>>>>>>>>>>>> again.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I am quite certain that jobmanager-2 does not know about any
>>>>>>>>>>>>>>> of jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>>>>>>>>>>>> duplicate job folders:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>>>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>>>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>>>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>>>>>>>>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>>>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in
>>>>>>>>>>>>>>> Zookeeper when the job is stopped, but fails since there are leftover files
>>>>>>>>>>>>>>> in it from jobmanager-1:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>>>>>>>>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>>>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>>>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I’ve noticed that when restoring the job, it seems like
>>>>>>>>>>>>>>> jobmanager-2 does not get anything more than jobID, while it perhaps needs
>>>>>>>>>>>>>>> some metadata? Here is the log that seems suspicious to me:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2018-08-27 13:09:18,113 INFO
>>>>>>>>>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>>>>>>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to
>>>>>>>>>>>>>>> be aware that it’s overwriting anything or not deleting properly.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> My question is - what is the intended way for the job
>>>>>>>>>>>>>>> managers to correctly exchange metadata in HA mode and why is it not
>>>>>>>>>>>>>>> working for me?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks in advance!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by Till Rohrmann <tr...@apache.org>.
Hi Encho,

thanks for sending me the logs. I think I found a bug which could explain
what you are observing: We listen to newly added jobs and try to recover
them independent of the leadership status. Due to this also a standby
JobManager tries to recover a submitted job but won't execute it.
Unfortunately, recovering a job already locks it without releasing the lock
if it cannot be executed. I've documented the problem here [1]. This is a
quite mean bug which we should fix asap. Thanks a lot for reporting the
problem!

[1] https://issues.apache.org/jira/browse/FLINK-10255.

Cheers,
Till

On Wed, Aug 29, 2018 at 3:31 PM Encho Mishinev <en...@gmail.com>
wrote:

> Hi Till,
>
> Those are actually the full logs except the two parts I shortened
> (pipeline construction and execution). As I said - accessing the UI for
> Jobmanager 2 redirects to Jobmanager 1 so it seems like he is aware that he
> is not the leader. Jobmanager 2 has no other logs than what I sent. Here is
> the full end-to-end log of Jobmanager 2 after repeating the experiment
> again:
>
> Starting Job Manager
> sed: cannot rename /opt/flink/conf/sediVa6XS: Device or resource busy
> config file:
> jobmanager.rpc.address: flink-jobmanager-2
> jobmanager.rpc.port: 6123
> jobmanager.heap.size: 8192
> taskmanager.heap.size: 8192
> taskmanager.numberOfTaskSlots: 4
> high-availability: zookeeper
> high-availability.storageDir:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
> high-availability.zookeeper.quorum: zk-cs:2181
> high-availability.zookeeper.path.root: /flink
> high-availability.jobmanager.port: 50010
> state.backend: filesystem
> state.checkpoints.dir:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
> state.savepoints.dir:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
> state.backend.incremental: false
> fs.default-scheme:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
> rest.port: 8081
> web.upload.dir: /opt/flink/upload
> query.server.port: 6125
> taskmanager.numberOfTaskSlots: 4
> classloader.parent-first-patterns.additional: org.apache.xerces.
> blob.storage.directory: /opt/flink/blob-server
> blob.server.port: 6124
> blob.server.port: 6124
> query.server.port: 6125
> Starting standalonesession as a console application on host
> flink-jobmanager-2-7844b78c9-zwdqv.
> 2018-08-29 13:19:24,047 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
> --------------------------------------------------------------------------------
> 2018-08-29 13:19:24,049 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
> StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
> Date:16.08.2018 @ 06:39:50 GMT)
> 2018-08-29 13:19:24,049 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
> user: flink
> 2018-08-29 13:19:24,367 WARN  org.apache.hadoop.util.NativeCodeLoader
>                  - Unable to load native-hadoop library for your
> platform... using builtin-java classes where applicable
> 2018-08-29 13:19:24,431 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
> Hadoop/Kerberos user: flink
> 2018-08-29 13:19:24,431 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
> 2018-08-29 13:19:24,431 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
> heap size: 6702 MiBytes
> 2018-08-29 13:19:24,431 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
> /docker-java-home/jre
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
> version: 2.7.5
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
> Options:
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
> Arguments:
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  --configDir
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  /opt/flink/conf
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  --executionMode
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
> 2018-08-29 13:19:24,434 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
> 2018-08-29 13:19:24,435 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
> /opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
> 2018-08-29 13:19:24,435 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
> --------------------------------------------------------------------------------
> 2018-08-29 13:19:24,436 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
> UNIX signal handlers for [TERM, HUP, INT]
> 2018-08-29 13:19:24,442 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: jobmanager.rpc.address, flink-jobmanager-2
> 2018-08-29 13:19:24,442 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: jobmanager.rpc.port, 6123
> 2018-08-29 13:19:24,442 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: jobmanager.heap.size, 8192
> 2018-08-29 13:19:24,442 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: taskmanager.heap.size, 8192
> 2018-08-29 13:19:24,442 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: taskmanager.numberOfTaskSlots, 4
> 2018-08-29 13:19:24,442 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability, zookeeper
> 2018-08-29 13:19:24,443 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.storageDir,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
> 2018-08-29 13:19:24,443 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.zookeeper.quorum, zk-cs:2181
> 2018-08-29 13:19:24,443 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.zookeeper.path.root, /flink
> 2018-08-29 13:19:24,443 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.jobmanager.port, 50010
> 2018-08-29 13:19:24,443 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.backend, filesystem
> 2018-08-29 13:19:24,443 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.checkpoints.dir,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
> 2018-08-29 13:19:24,444 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.savepoints.dir,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
> 2018-08-29 13:19:24,444 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.backend.incremental, false
> 2018-08-29 13:19:24,444 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: fs.default-scheme,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
> 2018-08-29 13:19:24,444 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: rest.port, 8081
> 2018-08-29 13:19:24,444 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: web.upload.dir, /opt/flink/upload
> 2018-08-29 13:19:24,444 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: query.server.port, 6125
> 2018-08-29 13:19:24,445 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: taskmanager.numberOfTaskSlots, 4
> 2018-08-29 13:19:24,445 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: classloader.parent-first-patterns.additional,
> org.apache.xerces.
> 2018-08-29 13:19:24,445 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: blob.storage.directory, /opt/flink/blob-server
> 2018-08-29 13:19:24,445 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: blob.server.port, 6124
> 2018-08-29 13:19:24,445 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: blob.server.port, 6124
> 2018-08-29 13:19:24,445 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: query.server.port, 6125
> 2018-08-29 13:19:24,461 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
> StandaloneSessionClusterEntrypoint.
> 2018-08-29 13:19:24,461 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
> default filesystem.
> 2018-08-29 13:19:24,472 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
> security context.
> 2018-08-29 13:19:24,506 INFO
> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
> set to flink (auth:SIMPLE)
> 2018-08-29 13:19:24,522 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
> Initializing cluster services.
> 2018-08-29 13:19:24,532 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
> start actor system at flink-jobmanager-2:50010
> 2018-08-29 13:19:24,996 INFO  akka.event.slf4j.Slf4jLogger
>                   - Slf4jLogger started
> 2018-08-29 13:19:25,050 INFO  akka.remote.Remoting
>                   - Starting remoting
> 2018-08-29 13:19:25,209 INFO  akka.remote.Remoting
>                   - Remoting started; listening on addresses
> :[akka.tcp://flink@flink-jobmanager-2:50010]
> 2018-08-29 13:19:25,216 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
> system started at akka.tcp://flink@flink-jobmanager-2:50010
> 2018-08-29 13:19:25,648 INFO
> org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
> highly available BLOB storage directory at
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
> 2018-08-29 13:19:25,702 INFO
> org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing
> default ACL for ZK connections
> 2018-08-29 13:19:25,703 INFO
> org.apache.flink.runtime.util.ZooKeeperUtils                  - Using
> '/flink/default' as Zookeeper namespace.
> 2018-08-29 13:19:25,750 INFO
> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
> - Starting
> 2018-08-29 13:19:25,756 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
> built on 03/23/2017 10:13 GMT
> 2018-08-29 13:19:25,756 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:host.name=flink-jobmanager-2-7844b78c9-zwdqv
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.version=1.8.0_181
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.vendor=Oracle Corporation
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.io.tmpdir=/tmp
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.compiler=<NA>
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:os.name=Linux
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:os.arch=amd64
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:os.version=4.4.0-1027-gke
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:user.name=flink
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:user.home=/opt/flink
> 2018-08-29 13:19:25,757 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:user.dir=/opt/flink
> 2018-08-29 13:19:25,758 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
> Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
> 2018-08-29 13:19:25,775 WARN
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
> configuration failed: javax.security.auth.login.LoginException: No JAAS
> configuration section named 'Client' was found in specified JAAS
> configuration file: '/tmp/jaas-5000339768628554676.conf'. Will continue
> connection to Zookeeper server without SASL authentication, if Zookeeper
> server allows it.
> 2018-08-29 13:19:25,776 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
> Opening socket connection to server zk-cs.default.svc.cluster.local/
> 10.27.248.104:2181
> 2018-08-29 13:19:25,777 ERROR
> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
> Authentication failed
> 2018-08-29 13:19:25,777 INFO  org.apache.flink.runtime.blob.BlobServer
>                   - Created BLOB server storage directory
> /opt/flink/blob-server/blobStore-40cefeee-e8d1-4522-aea3-957d9f7fbeee
> 2018-08-29 13:19:25,777 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
> connection established to zk-cs.default.svc.cluster.local/
> 10.27.248.104:2181, initiating session
> 2018-08-29 13:19:25,778 INFO  org.apache.flink.runtime.blob.BlobServer
>                   - Started BLOB server at 0.0.0.0:6124 - max concurrent
> requests: 50 - max backlog: 1000
> 2018-08-29 13:19:25,788 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
> Session establishment complete on server zk-cs.default.svc.cluster.local/
> 10.27.248.104:2181, sessionid = 0x26584fd55690009, negotiated timeout =
> 40000
> 2018-08-29 13:19:25,789 INFO
> org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
> - State change: CONNECTED
> 2018-08-29 13:19:25,793 INFO
> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
> reporter configured, no metrics will be exposed/reported.
> 2018-08-29 13:19:25,798 INFO
> org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
> Initializing FileArchivedExecutionGraphStore: Storage directory
> /tmp/executionGraphStore-76cce4e7-84ea-4624-a847-bbd7fdc4f109, expiration
> time 3600000, maximum cache size 52428800 bytes.
> 2018-08-29 13:19:25,824 INFO
> org.apache.flink.runtime.blob.TransientBlobCache              - Created
> BLOB cache storage directory
> /opt/flink/blob-server/blobStore-7c6c2db0-f7ab-4cb6-909d-6c9cbfd78215
> 2018-08-29 13:19:25,838 WARN
> org.apache.flink.configuration.Configuration                  - Config uses
> deprecated configuration key 'jobmanager.rpc.address' instead of proper key
> 'rest.address'
> 2018-08-29 13:19:25,839 WARN
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
> directory /opt/flink/upload/flink-web-upload does not exist, or has been
> deleted externally. Previously uploaded files are no longer available.
> 2018-08-29 13:19:25,840 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
> directory /opt/flink/upload/flink-web-upload for file uploads.
> 2018-08-29 13:19:25,843 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
> rest endpoint.
> 2018-08-29 13:19:26,143 WARN
> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
> environment variable 'log.file' is not set.
> 2018-08-29 13:19:26,143 WARN
> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
> log files are unavailable in the web dashboard. Log file location not found
> in environment variable 'log.file' or configuration key 'Key:
> 'web.log.path' , default: null (deprecated keys:
> [jobmanager.web.log.path])'.
> 2018-08-29 13:19:26,216 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
> endpoint listening at flink-jobmanager-2:8081
> 2018-08-29 13:19:26,217 INFO
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
> Starting ZooKeeperLeaderElectionService
> ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
> 2018-08-29 13:19:26,236 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
> frontend listening at http://flink-jobmanager-2:8081.
> 2018-08-29 13:19:26,248 INFO
> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
> RPC endpoint for
> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
> akka://flink/user/resourcemanager .
> 2018-08-29 13:19:26,323 INFO
> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
> RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
> at akka://flink/user/dispatcher .
> 2018-08-29 13:19:26,335 INFO
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
> Starting ZooKeeperLeaderElectionService
> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
> 2018-08-29 13:19:26,336 INFO
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
> Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
> 2018-08-29 13:19:26,338 INFO
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
> Starting ZooKeeperLeaderElectionService
> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
> 2018-08-29 13:19:26,339 INFO
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
> 2018-08-29 13:23:21,513 INFO
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
> Recovered SubmittedJobGraph(836b29f7c66bbeb6ed8bae41cb9b316c, null).
>
> Thanks,
> Encho
>
> On Wed, Aug 29, 2018 at 3:59 PM Till Rohrmann <tr...@apache.org>
> wrote:
>
>> Hi Encho,
>>
>> thanks for sending the first part of the logs. What I would actually be
>> interested in are the complete logs because somewhere in the jobmanager-2
>> logs there must be a log statement saying that the respective dispatcher
>> gained leadership. I would like to see why this happens but for this to
>> debug the complete logs are necessary. It would be awesome if you could
>> send them to me. Thanks a lot!
>>
>> Cheers,
>> Till
>>
>> On Wed, Aug 29, 2018 at 2:00 PM Encho Mishinev <en...@gmail.com>
>> wrote:
>>
>>> Hi Till,
>>>
>>> I will use the approach with a k8s deployment and HA mode with a single
>>> job manager. Nonetheless, here are the logs I just produced by repeating
>>> the aforementioned experiment, hope they help in debugging:
>>>
>>> *- Starting Jobmanager-1:*
>>>
>>> Starting Job Manager
>>> sed: cannot rename /opt/flink/conf/sedR98XPn: Device or resource busy
>>> config file:
>>> jobmanager.rpc.address: flink-jobmanager-1
>>> jobmanager.rpc.port: 6123
>>> jobmanager.heap.size: 8192
>>> taskmanager.heap.size: 8192
>>> taskmanager.numberOfTaskSlots: 4
>>> high-availability: zookeeper
>>> high-availability.storageDir:
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>>> high-availability.zookeeper.quorum: zk-cs:2181
>>> high-availability.zookeeper.path.root: /flink
>>> high-availability.jobmanager.port: 50010
>>> state.backend: filesystem
>>> state.checkpoints.dir:
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>>> state.savepoints.dir:
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>>> state.backend.incremental: false
>>> fs.default-scheme:
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>>> rest.port: 8081
>>> web.upload.dir: /opt/flink/upload
>>> query.server.port: 6125
>>> taskmanager.numberOfTaskSlots: 4
>>> classloader.parent-first-patterns.additional: org.apache.xerces.
>>> blob.storage.directory: /opt/flink/blob-server
>>> blob.server.port: 6124
>>> blob.server.port: 6124
>>> query.server.port: 6125
>>> Starting standalonesession as a console application on host
>>> flink-jobmanager-1-f76fd4df8-ftwt9.
>>> 2018-08-29 11:41:48,806 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>> --------------------------------------------------------------------------------
>>> 2018-08-29 11:41:48,807 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
>>> StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
>>> Date:16.08.2018 @ 06:39:50 GMT)
>>> 2018-08-29 11:41:48,807 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
>>> user: flink
>>> 2018-08-29 11:41:49,134 WARN  org.apache.hadoop.util.NativeCodeLoader
>>>                    - Unable to load native-hadoop library for your
>>> platform... using builtin-java classes where applicable
>>> 2018-08-29 11:41:49,210 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
>>> Hadoop/Kerberos user: flink
>>> 2018-08-29 11:41:49,210 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
>>> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
>>> 2018-08-29 11:41:49,210 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
>>> heap size: 6702 MiBytes
>>> 2018-08-29 11:41:49,210 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
>>> /docker-java-home/jre
>>> 2018-08-29 11:41:49,213 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
>>> version: 2.7.5
>>> 2018-08-29 11:41:49,213 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
>>> Options:
>>> 2018-08-29 11:41:49,213 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>  -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
>>> 2018-08-29 11:41:49,213 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>  -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
>>> 2018-08-29 11:41:49,213 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
>>> Arguments:
>>> 2018-08-29 11:41:49,213 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>  --configDir
>>> 2018-08-29 11:41:49,213 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>  /opt/flink/conf
>>> 2018-08-29 11:41:49,213 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>  --executionMode
>>> 2018-08-29 11:41:49,213 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>>> 2018-08-29 11:41:49,214 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
>>> 2018-08-29 11:41:49,214 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>>> 2018-08-29 11:41:49,214 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
>>> /opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>>> 2018-08-29 11:41:49,214 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>> --------------------------------------------------------------------------------
>>> 2018-08-29 11:41:49,215 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
>>> UNIX signal handlers for [TERM, HUP, INT]
>>> 2018-08-29 11:41:49,221 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: jobmanager.rpc.address, flink-jobmanager-1
>>> 2018-08-29 11:41:49,221 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: jobmanager.rpc.port, 6123
>>> 2018-08-29 11:41:49,221 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: jobmanager.heap.size, 8192
>>> 2018-08-29 11:41:49,221 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: taskmanager.heap.size, 8192
>>> 2018-08-29 11:41:49,221 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: taskmanager.numberOfTaskSlots, 4
>>> 2018-08-29 11:41:49,222 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: high-availability, zookeeper
>>> 2018-08-29 11:41:49,222 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: high-availability.storageDir,
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>>> 2018-08-29 11:41:49,222 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: high-availability.zookeeper.quorum, zk-cs:2181
>>> 2018-08-29 11:41:49,222 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: high-availability.zookeeper.path.root, /flink
>>> 2018-08-29 11:41:49,223 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: high-availability.jobmanager.port, 50010
>>> 2018-08-29 11:41:49,223 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: state.backend, filesystem
>>> 2018-08-29 11:41:49,223 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: state.checkpoints.dir,
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>>> 2018-08-29 11:41:49,223 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: state.savepoints.dir,
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>>> 2018-08-29 11:41:49,223 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: state.backend.incremental, false
>>> 2018-08-29 11:41:49,224 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: fs.default-scheme,
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>>> 2018-08-29 11:41:49,224 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: rest.port, 8081
>>> 2018-08-29 11:41:49,224 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: web.upload.dir, /opt/flink/upload
>>> 2018-08-29 11:41:49,224 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: query.server.port, 6125
>>> 2018-08-29 11:41:49,225 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: taskmanager.numberOfTaskSlots, 4
>>> 2018-08-29 11:41:49,225 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: classloader.parent-first-patterns.additional,
>>> org.apache.xerces.
>>> 2018-08-29 11:41:49,225 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: blob.storage.directory, /opt/flink/blob-server
>>> 2018-08-29 11:41:49,225 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: blob.server.port, 6124
>>> 2018-08-29 11:41:49,225 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: blob.server.port, 6124
>>> 2018-08-29 11:41:49,225 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: query.server.port, 6125
>>> 2018-08-29 11:41:49,239 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
>>> StandaloneSessionClusterEntrypoint.
>>> 2018-08-29 11:41:49,239 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>>> default filesystem.
>>> 2018-08-29 11:41:49,250 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>>> security context.
>>> 2018-08-29 11:41:49,282 INFO
>>> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
>>> set to flink (auth:SIMPLE)
>>> 2018-08-29 11:41:49,298 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>> Initializing cluster services.
>>> 2018-08-29 11:41:49,309 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
>>> start actor system at flink-jobmanager-1:50010
>>> 2018-08-29 11:41:49,768 INFO  akka.event.slf4j.Slf4jLogger
>>>                     - Slf4jLogger started
>>> 2018-08-29 11:41:49,823 INFO  akka.remote.Remoting
>>>                     - Starting remoting
>>> 2018-08-29 11:41:49,974 INFO  akka.remote.Remoting
>>>                     - Remoting started; listening on addresses
>>> :[akka.tcp://flink@flink-jobmanager-1:50010]
>>> 2018-08-29 11:41:49,981 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
>>> system started at akka.tcp://flink@flink-jobmanager-1:50010
>>> 2018-08-29 11:41:50,444 INFO
>>> org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
>>> highly available BLOB storage directory at
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
>>> 2018-08-29 11:41:50,509 INFO
>>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing
>>> default ACL for ZK connections
>>> 2018-08-29 11:41:50,509 INFO
>>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Using
>>> '/flink/default' as Zookeeper namespace.
>>> 2018-08-29 11:41:50,568 INFO
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
>>> - Starting
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
>>> built on 03/23/2017 10:13 GMT
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:host.name=flink-jobmanager-1-f76fd4df8-ftwt9
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.version=1.8.0_181
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.vendor=Oracle Corporation
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.io.tmpdir=/tmp
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.compiler=<NA>
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:os.name=Linux
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:os.arch=amd64
>>> 2018-08-29 11:41:50,577 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:os.version=4.4.0-1027-gke
>>> 2018-08-29 11:41:50,578 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:user.name=flink
>>> 2018-08-29 11:41:50,578 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:user.home=/opt/flink
>>> 2018-08-29 11:41:50,578 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:user.dir=/opt/flink
>>> 2018-08-29 11:41:50,578 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
>>> Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
>>> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
>>> 2018-08-29 11:41:50,605 INFO  org.apache.flink.runtime.blob.BlobServer
>>>                     - Created BLOB server storage directory
>>> /opt/flink/blob-server/blobStore-d408cea8-2ed0-461a-a30a-a62b70fd332a
>>> 2018-08-29 11:41:50,605 WARN
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
>>> configuration failed: javax.security.auth.login.LoginException: No JAAS
>>> configuration section named 'Client' was found in specified JAAS
>>> configuration file: '/tmp/jaas-5372401662150571998.conf'. Will continue
>>> connection to Zookeeper server without SASL authentication, if Zookeeper
>>> server allows it.
>>> 2018-08-29 11:41:50,607 INFO  org.apache.flink.runtime.blob.BlobServer
>>>                     - Started BLOB server at 0.0.0.0:6124 - max
>>> concurrent requests: 50 - max backlog: 1000
>>> 2018-08-29 11:41:50,607 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>>> Opening socket connection to server zk-cs.default.svc.cluster.local/
>>> 10.27.248.104:2181
>>> 2018-08-29 11:41:50,608 ERROR
>>> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
>>> Authentication failed
>>> 2018-08-29 11:41:50,609 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
>>> connection established to zk-cs.default.svc.cluster.local/
>>> 10.27.248.104:2181, initiating session
>>> 2018-08-29 11:41:50,618 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>>> Session establishment complete on server zk-cs.default.svc.cluster.local/
>>> 10.27.248.104:2181, sessionid = 0x26584fd55690005, negotiated timeout =
>>> 40000
>>> 2018-08-29 11:41:50,619 INFO
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
>>> - State change: CONNECTED
>>> 2018-08-29 11:41:50,627 INFO
>>> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
>>> reporter configured, no metrics will be exposed/reported.
>>> 2018-08-29 11:41:50,633 INFO
>>> org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
>>> Initializing FileArchivedExecutionGraphStore: Storage directory
>>> /tmp/executionGraphStore-c5df0b39-86f3-4fba-bdda-aacca4f86086, expiration
>>> time 3600000, maximum cache size 52428800 bytes.
>>> 2018-08-29 11:41:50,659 INFO
>>> org.apache.flink.runtime.blob.TransientBlobCache              - Created
>>> BLOB cache storage directory
>>> /opt/flink/blob-server/blobStore-c12d55af-3c2d-4fc2-8ee8-6de642522184
>>> 2018-08-29 11:41:50,674 WARN
>>> org.apache.flink.configuration.Configuration                  - Config uses
>>> deprecated configuration key 'jobmanager.rpc.address' instead of proper key
>>> 'rest.address'
>>> 2018-08-29 11:41:50,675 WARN
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
>>> directory /opt/flink/upload/flink-web-upload does not exist, or has been
>>> deleted externally. Previously uploaded files are no longer available.
>>> 2018-08-29 11:41:50,676 INFO
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
>>> directory /opt/flink/upload/flink-web-upload for file uploads.
>>> 2018-08-29 11:41:50,679 INFO
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
>>> rest endpoint.
>>> 2018-08-29 11:41:50,995 WARN
>>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
>>> environment variable 'log.file' is not set.
>>> 2018-08-29 11:41:50,995 WARN
>>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
>>> log files are unavailable in the web dashboard. Log file location not found
>>> in environment variable 'log.file' or configuration key 'Key:
>>> 'web.log.path' , default: null (deprecated keys:
>>> [jobmanager.web.log.path])'.
>>> 2018-08-29 11:41:51,071 INFO
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
>>> endpoint listening at flink-jobmanager-1:8081
>>> 2018-08-29 11:41:51,071 INFO
>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>> Starting ZooKeeperLeaderElectionService
>>> ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
>>> 2018-08-29 11:41:51,091 WARN
>>> org.apache.flink.shaded.curator.org.apache.curator.utils.ZKPaths  - The
>>> version of ZooKeeper being used doesn't support Container nodes.
>>> CreateMode.PERSISTENT will be used instead.
>>> 2018-08-29 11:41:51,101 INFO
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
>>> frontend listening at http://flink-jobmanager-1:8081.
>>> 2018-08-29 11:41:51,114 INFO
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>>> RPC endpoint for
>>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
>>> akka://flink/user/resourcemanager .
>>> 2018-08-29 11:41:51,141 INFO
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    -
>>> http://flink-jobmanager-1:8081 was granted leadership with
>>> leaderSessionID=bb0d4dfd-c2c4-480b-bc86-62e231a606dd
>>> 2018-08-29 11:41:51,214 INFO
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>>> RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
>>> at akka://flink/user/dispatcher .
>>> 2018-08-29 11:41:51,230 INFO
>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>> Starting ZooKeeperLeaderElectionService
>>> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
>>> 2018-08-29 11:41:51,232 INFO
>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>> Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>>> 2018-08-29 11:41:51,234 INFO
>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>> Starting ZooKeeperLeaderElectionService
>>> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
>>> 2018-08-29 11:41:51,235 INFO
>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>> 2018-08-29 11:41:51,253 INFO
>>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  -
>>> ResourceManager akka.tcp://flink@flink-jobmanager-1:50010/user/resourcemanager
>>> was granted leadership with fencing token ba47ed8daa8ff16bea6fc355c13f4d49
>>> 2018-08-29 11:41:51,254 INFO
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
>>> Starting the SlotManager.
>>> 2018-08-29 11:41:51,263 INFO
>>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Dispatcher
>>> akka.tcp://flink@flink-jobmanager-1:50010/user/dispatcher was granted
>>> leadership with fencing token 703301bf-85e7-4464-990f-ad39128a7b4d
>>> 2018-08-29 11:41:51,263 INFO
>>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Recovering
>>> all persisted jobs.
>>> 2018-08-29 11:41:51,468 INFO
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
>>> Registering TaskManager c8a3201d58d87dbbe16f8eb352b5c5b6 under
>>> 1c5bf0bc3848bd384b6f032ff7213754 at the SlotManager.
>>> 2018-08-29 11:41:51,471 INFO
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
>>> Registering TaskManager 104d18b72fed054620e58e120a1ea083 under
>>> e9d3e8ad3b477dd2e58bcb88a2c0d061 at the SlotManager.
>>>
>>> *Starting Jobmanager-2:*
>>>
>>> Starting Job Manager
>>> sed: cannot rename /opt/flink/conf/sedH2ZiSu: Device or resource busy
>>> config file:
>>> jobmanager.rpc.address: flink-jobmanager-2
>>> jobmanager.rpc.port: 6123
>>> jobmanager.heap.size: 8192
>>> taskmanager.heap.size: 8192
>>> taskmanager.numberOfTaskSlots: 4
>>> high-availability: zookeeper
>>> high-availability.storageDir:
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>>> high-availability.zookeeper.quorum: zk-cs:2181
>>> high-availability.zookeeper.path.root: /flink
>>> high-availability.jobmanager.port: 50010
>>> state.backend: filesystem
>>> state.checkpoints.dir:
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>>> state.savepoints.dir:
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>>> state.backend.incremental: false
>>> fs.default-scheme:
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>>> rest.port: 8081
>>> web.upload.dir: /opt/flink/upload
>>> query.server.port: 6125
>>> taskmanager.numberOfTaskSlots: 4
>>> classloader.parent-first-patterns.additional: org.apache.xerces.
>>> blob.storage.directory: /opt/flink/blob-server
>>> blob.server.port: 6124
>>> blob.server.port: 6124
>>> query.server.port: 6125
>>> Starting standalonesession as a console application on host
>>> flink-jobmanager-2-7844b78c9-kmvw9.
>>> 2018-08-29 11:41:51,688 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>> --------------------------------------------------------------------------------
>>> 2018-08-29 11:41:51,690 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
>>> StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
>>> Date:16.08.2018 @ 06:39:50 GMT)
>>> 2018-08-29 11:41:51,690 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
>>> user: flink
>>> 2018-08-29 11:41:52,018 WARN  org.apache.hadoop.util.NativeCodeLoader
>>>                    - Unable to load native-hadoop library for your
>>> platform... using builtin-java classes where applicable
>>> 2018-08-29 11:41:52,088 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
>>> Hadoop/Kerberos user: flink
>>> 2018-08-29 11:41:52,088 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
>>> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
>>> 2018-08-29 11:41:52,088 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
>>> heap size: 6702 MiBytes
>>> 2018-08-29 11:41:52,088 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
>>> /docker-java-home/jre
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
>>> version: 2.7.5
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
>>> Options:
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>  -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>  -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
>>> Arguments:
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>  --configDir
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>  /opt/flink/conf
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>  --executionMode
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
>>> /opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>>> 2018-08-29 11:41:52,091 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>> --------------------------------------------------------------------------------
>>> 2018-08-29 11:41:52,092 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
>>> UNIX signal handlers for [TERM, HUP, INT]
>>> 2018-08-29 11:41:52,103 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: jobmanager.rpc.address, flink-jobmanager-2
>>> 2018-08-29 11:41:52,103 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: jobmanager.rpc.port, 6123
>>> 2018-08-29 11:41:52,103 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: jobmanager.heap.size, 8192
>>> 2018-08-29 11:41:52,104 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: taskmanager.heap.size, 8192
>>> 2018-08-29 11:41:52,104 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: taskmanager.numberOfTaskSlots, 4
>>> 2018-08-29 11:41:52,104 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: high-availability, zookeeper
>>> 2018-08-29 11:41:52,104 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: high-availability.storageDir,
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>>> 2018-08-29 11:41:52,104 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: high-availability.zookeeper.quorum, zk-cs:2181
>>> 2018-08-29 11:41:52,104 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: high-availability.zookeeper.path.root, /flink
>>> 2018-08-29 11:41:52,105 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: high-availability.jobmanager.port, 50010
>>> 2018-08-29 11:41:52,105 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: state.backend, filesystem
>>> 2018-08-29 11:41:52,105 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: state.checkpoints.dir,
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>>> 2018-08-29 11:41:52,105 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: state.savepoints.dir,
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>>> 2018-08-29 11:41:52,105 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: state.backend.incremental, false
>>> 2018-08-29 11:41:52,106 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: fs.default-scheme,
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>>> 2018-08-29 11:41:52,106 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: rest.port, 8081
>>> 2018-08-29 11:41:52,106 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: web.upload.dir, /opt/flink/upload
>>> 2018-08-29 11:41:52,106 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: query.server.port, 6125
>>> 2018-08-29 11:41:52,106 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: taskmanager.numberOfTaskSlots, 4
>>> 2018-08-29 11:41:52,107 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: classloader.parent-first-patterns.additional,
>>> org.apache.xerces.
>>> 2018-08-29 11:41:52,107 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: blob.storage.directory, /opt/flink/blob-server
>>> 2018-08-29 11:41:52,107 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: blob.server.port, 6124
>>> 2018-08-29 11:41:52,107 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: blob.server.port, 6124
>>> 2018-08-29 11:41:52,107 INFO
>>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>>> configuration property: query.server.port, 6125
>>> 2018-08-29 11:41:52,122 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
>>> StandaloneSessionClusterEntrypoint.
>>> 2018-08-29 11:41:52,123 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>>> default filesystem.
>>> 2018-08-29 11:41:52,133 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>>> security context.
>>> 2018-08-29 11:41:52,173 INFO
>>> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
>>> set to flink (auth:SIMPLE)
>>> 2018-08-29 11:41:52,188 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>> Initializing cluster services.
>>> 2018-08-29 11:41:52,198 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
>>> start actor system at flink-jobmanager-2:50010
>>> 2018-08-29 11:41:52,753 INFO  akka.event.slf4j.Slf4jLogger
>>>                     - Slf4jLogger started
>>> 2018-08-29 11:41:52,822 INFO  akka.remote.Remoting
>>>                     - Starting remoting
>>> 2018-08-29 11:41:53,038 INFO  akka.remote.Remoting
>>>                     - Remoting started; listening on addresses
>>> :[akka.tcp://flink@flink-jobmanager-2:50010]
>>> 2018-08-29 11:41:53,046 INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
>>> system started at akka.tcp://flink@flink-jobmanager-2:50010
>>> 2018-08-29 11:41:53,500 INFO
>>> org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
>>> highly available BLOB storage directory at
>>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
>>> 2018-08-29 11:41:53,558 INFO
>>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing
>>> default ACL for ZK connections
>>> 2018-08-29 11:41:53,559 INFO
>>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Using
>>> '/flink/default' as Zookeeper namespace.
>>> 2018-08-29 11:41:53,616 INFO
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
>>> - Starting
>>> 2018-08-29 11:41:53,624 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
>>> built on 03/23/2017 10:13 GMT
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:host.name=flink-jobmanager-2-7844b78c9-kmvw9
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.version=1.8.0_181
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.vendor=Oracle Corporation
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.io.tmpdir=/tmp
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.compiler=<NA>
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:os.name=Linux
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:os.arch=amd64
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:os.version=4.4.0-1027-gke
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:user.name=flink
>>> 2018-08-29 11:41:53,625 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:user.home=/opt/flink
>>> 2018-08-29 11:41:53,626 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:user.dir=/opt/flink
>>> 2018-08-29 11:41:53,626 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
>>> Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
>>> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
>>> 2018-08-29 11:41:53,644 WARN
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
>>> configuration failed: javax.security.auth.login.LoginException: No JAAS
>>> configuration section named 'Client' was found in specified JAAS
>>> configuration file: '/tmp/jaas-8238466329925822361.conf'. Will continue
>>> connection to Zookeeper server without SASL authentication, if Zookeeper
>>> server allows it.
>>> 2018-08-29 11:41:53,646 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>>> Opening socket connection to server zk-cs.default.svc.cluster.local/
>>> 10.27.248.104:2181
>>> 2018-08-29 11:41:53,646 INFO  org.apache.flink.runtime.blob.BlobServer
>>>                     - Created BLOB server storage directory
>>> /opt/flink/blob-server/blobStore-61cdb645-5d0c-47fd-bcf6-84ad16fadade
>>> 2018-08-29 11:41:53,646 ERROR
>>> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
>>> Authentication failed
>>> 2018-08-29 11:41:53,647 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
>>> connection established to zk-cs.default.svc.cluster.local/
>>> 10.27.248.104:2181, initiating session
>>> 2018-08-29 11:41:53,649 INFO  org.apache.flink.runtime.blob.BlobServer
>>>                     - Started BLOB server at 0.0.0.0:6124 - max
>>> concurrent requests: 50 - max backlog: 1000
>>> 2018-08-29 11:41:53,655 INFO
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>>> Session establishment complete on server zk-cs.default.svc.cluster.local/
>>> 10.27.248.104:2181, sessionid = 0x26584fd55690006, negotiated timeout =
>>> 40000
>>> 2018-08-29 11:41:53,656 INFO
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
>>> - State change: CONNECTED
>>> 2018-08-29 11:41:53,667 INFO
>>> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
>>> reporter configured, no metrics will be exposed/reported.
>>> 2018-08-29 11:41:53,673 INFO
>>> org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
>>> Initializing FileArchivedExecutionGraphStore: Storage directory
>>> /tmp/executionGraphStore-8b236c14-79ee-4a84-b23f-437408c4661a, expiration
>>> time 3600000, maximum cache size 52428800 bytes.
>>> 2018-08-29 11:41:53,699 INFO
>>> org.apache.flink.runtime.blob.TransientBlobCache              - Created
>>> BLOB cache storage directory
>>> /opt/flink/blob-server/blobStore-80c519df-cc6f-4e9c-9cd5-da4077c826f0
>>> 2018-08-29 11:41:53,717 WARN
>>> org.apache.flink.configuration.Configuration                  - Config uses
>>> deprecated configuration key 'jobmanager.rpc.address' instead of proper key
>>> 'rest.address'
>>> 2018-08-29 11:41:53,718 WARN
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
>>> directory /opt/flink/upload/flink-web-upload does not exist, or has been
>>> deleted externally. Previously uploaded files are no longer available.
>>> 2018-08-29 11:41:53,719 INFO
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
>>> directory /opt/flink/upload/flink-web-upload for file uploads.
>>> 2018-08-29 11:41:53,722 INFO
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
>>> rest endpoint.
>>> 2018-08-29 11:41:54,084 WARN
>>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
>>> environment variable 'log.file' is not set.
>>> 2018-08-29 11:41:54,084 WARN
>>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
>>> log files are unavailable in the web dashboard. Log file location not found
>>> in environment variable 'log.file' or configuration key 'Key:
>>> 'web.log.path' , default: null (deprecated keys:
>>> [jobmanager.web.log.path])'.
>>> 2018-08-29 11:41:54,160 INFO
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
>>> endpoint listening at flink-jobmanager-2:8081
>>> 2018-08-29 11:41:54,160 INFO
>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>> Starting ZooKeeperLeaderElectionService
>>> ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
>>> 2018-08-29 11:41:54,180 INFO
>>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
>>> frontend listening at http://flink-jobmanager-2:8081.
>>> 2018-08-29 11:41:54,192 INFO
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>>> RPC endpoint for
>>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
>>> akka://flink/user/resourcemanager .
>>> 2018-08-29 11:41:54,273 INFO
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>>> RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
>>> at akka://flink/user/dispatcher .
>>> 2018-08-29 11:41:54,286 INFO
>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>> Starting ZooKeeperLeaderElectionService
>>> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
>>> 2018-08-29 11:41:54,287 INFO
>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>> Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>>> 2018-08-29 11:41:54,289 INFO
>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>> Starting ZooKeeperLeaderElectionService
>>> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
>>> 2018-08-29 11:41:54,289 INFO
>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>>
>>> *Upon submitting a batch job on Jobmanager-1, we immediately get this
>>> log on Jobmanager-2*
>>> 2018-08-29 11:47:06,249 INFO
>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>> Recovered SubmittedJobGraph(d69b67e4d28a2d244b06d3f6d661bca1, null).
>>>
>>> *Meanwhile Jobmanager-1 gets:*
>>> *-FlinkBatchPipelineTranslator pipeline logs- (we use Apache Beam)*
>>>
>>> 2018-08-29 11:47:06,006 INFO
>>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Submitting
>>> job d69b67e4d28a2d244b06d3f6d661bca1
>>> (sicassandrawriterbeam-flink-0829114703-7d95fabd).
>>> 2018-08-29 11:47:06,090 INFO
>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>> Added SubmittedJobGraph(d69b67e4d28a2d244b06d3f6d661bca1, null) to
>>> ZooKeeper.
>>>
>>> *-loads of job execution info-*
>>>
>>> 2018-08-29 11:49:20,272 INFO
>>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Job
>>> d69b67e4d28a2d244b06d3f6d661bca1 reached globally terminal state FINISHED.
>>> 2018-08-29 11:49:20,286 INFO
>>> org.apache.flink.runtime.jobmaster.JobMaster                  - Stopping
>>> the JobMaster for job
>>> sicassandrawriterbeam-flink-0829114703-7d95fabd(d69b67e4d28a2d244b06d3f6d661bca1).
>>> 2018-08-29 11:49:20,290 INFO
>>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>> Stopping ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>>> 2018-08-29 11:49:20,292 INFO
>>> org.apache.flink.runtime.jobmaster.JobMaster                  - Close
>>> ResourceManager connection 827b94881bf7c94d8516907e04e3a564: JobManager is
>>> shutting down..
>>> 2018-08-29 11:49:20,292 INFO
>>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Suspending
>>> SlotPool.
>>> 2018-08-29 11:49:20,293 INFO
>>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Stopping
>>> SlotPool.
>>> 2018-08-29 11:49:20,293 INFO
>>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  -
>>> Disconnect job manager a3dab0a0883c5f0f37943358d9104d79
>>> @akka.tcp://flink@flink-jobmanager-1:50010/user/jobmanager_0 for job
>>> d69b67e4d28a2d244b06d3f6d661bca1 from the resource manager.
>>> 2018-08-29 11:49:20,293 INFO
>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>> Stopping ZooKeeperLeaderElectionService
>>> ZooKeeperLeaderElectionService{leaderPath='/leader/d69b67e4d28a2d244b06d3f6d661bca1/job_manager_lock'}.
>>> 2018-08-29 11:49:20,304 INFO
>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>> Removed job graph d69b67e4d28a2d244b06d3f6d661bca1 from ZooKeeper.
>>>
>>>
>>> -------------------
>>>
>>> The result is:
>>> HDFS has only a jobgraph and an empty default folder - everything else
>>> is cleared
>>> ZooKeeper has the jobgraph that Jobmanager-1 claims to have removed in
>>> the last log still there.
>>>
>>> On Wed, Aug 29, 2018 at 12:14 PM Till Rohrmann <tr...@apache.org>
>>> wrote:
>>>
>>>> Hi Encho,
>>>>
>>>> it sounds strange that the standby JobManager tries to recover a
>>>> submitted job graph. This should only happen if it has been granted
>>>> leadership. Thus, it seems as if the standby JobManager thinks that it is
>>>> also the leader. Could you maybe share the logs of the two
>>>> JobManagers/ClusterEntrypoints with us?
>>>>
>>>> Running only a single JobManager/ClusterEntrypoint in HA mode via a
>>>> Kubernetes Deployment should do the trick and there is nothing wrong with
>>>> it.
>>>>
>>>> Cheers,
>>>> Till
>>>>
>>>> On Wed, Aug 29, 2018 at 11:05 AM Encho Mishinev <
>>>> encho.mishinev@gmail.com> wrote:
>>>>
>>>>> Hello,
>>>>>
>>>>> Since two job managers don't seem to be working for me I was thinking
>>>>> of just using a single job manager in Kubernetes in HA mode with a
>>>>> deployment ensuring its restart whenever it fails. Is this approach viable?
>>>>> The High-Availability page mentions that you use only one job manager in an
>>>>> YARN cluster but does not specify such option for Kubernetes. Is there
>>>>> anything that can go wrong with this approach?
>>>>>
>>>>> Thanks
>>>>>
>>>>> On Wed, Aug 29, 2018 at 11:10 AM Encho Mishinev <
>>>>> encho.mishinev@gmail.com> wrote:
>>>>>
>>>>>> Hi,
>>>>>>
>>>>>> Unfortunately the thing I described does indeed happen every time. As
>>>>>> mentioned in the first email, I am running on Kubernetes so certain things
>>>>>> could be different compared to just a standalone cluster.
>>>>>>
>>>>>> Any ideas for workarounds are welcome, as this problem basically
>>>>>> prevents me from using HA.
>>>>>>
>>>>>> Thanks,
>>>>>> Encho
>>>>>>
>>>>>> On Wed, Aug 29, 2018 at 5:15 AM vino yang <ya...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi Encho,
>>>>>>>
>>>>>>> From your description, I feel that there are extra bugs.
>>>>>>>
>>>>>>> About your description:
>>>>>>>
>>>>>>> *- Start both job managers*
>>>>>>> *- Start a batch job in JobManager 1 and let it finish*
>>>>>>> *The jobgraphs in both Zookeeper and HDFS remained.*
>>>>>>>
>>>>>>> Is it necessarily happening every time?
>>>>>>>
>>>>>>> In the Standalone cluster, the problems we encountered were sporadic.
>>>>>>>
>>>>>>> Thanks, vino.
>>>>>>>
>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午8:07写道:
>>>>>>>
>>>>>>>> Hello Till,
>>>>>>>>
>>>>>>>> I spend a few more hours testing and looking at the logs and it
>>>>>>>> seems like there's a more general problem here. While the two job managers
>>>>>>>> are active neither of them can properly delete jobgraphs. The above problem
>>>>>>>> I described comes from the fact that Kubernetes gets JobManager 1 quickly
>>>>>>>> after I manually kill it, so when I stop the job on JobManager 2 both are
>>>>>>>> alive.
>>>>>>>>
>>>>>>>> I did a very simple test:
>>>>>>>>
>>>>>>>> - Start both job managers
>>>>>>>> - Start a batch job in JobManager 1 and let it finish
>>>>>>>> The jobgraphs in both Zookeeper and HDFS remained.
>>>>>>>>
>>>>>>>> On the other hand if we do:
>>>>>>>>
>>>>>>>> - Start only JobManager 1 (again in HA mode)
>>>>>>>> - Start a batch job and let it finish
>>>>>>>> The jobgraphs in both Zookeeper and HDFS are deleted fine.
>>>>>>>>
>>>>>>>> It seems like the standby manager still leaves some kind of lock on
>>>>>>>> the jobgraphs. Do you think that's possible? Have you seen a similar
>>>>>>>> problem?
>>>>>>>> The only logs that appear on the standby manager while waiting are
>>>>>>>> of the type:
>>>>>>>>
>>>>>>>> 2018-08-28 11:54:10,789 INFO
>>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>>> Recovered SubmittedJobGraph(9e0a109b57511930c95d3b54574a66e3, null).
>>>>>>>>
>>>>>>>> Note that this log appears on the standby jobmanager immediately
>>>>>>>> when a new job is submitted to the active jobmanager.
>>>>>>>> Also note that the blobs and checkpoints are cleared fine. The
>>>>>>>> problem is only for jobgraphs both in ZooKeeper and HDFS.
>>>>>>>>
>>>>>>>> Trying to access the UI of the standby manager redirects to the
>>>>>>>> active one, so it is not a problem of them not knowing who the leader is.
>>>>>>>> Do you have any ideas?
>>>>>>>>
>>>>>>>> Thanks a lot,
>>>>>>>> Encho
>>>>>>>>
>>>>>>>> On Tue, Aug 28, 2018 at 10:27 AM Till Rohrmann <
>>>>>>>> trohrmann@apache.org> wrote:
>>>>>>>>
>>>>>>>>> Hi Encho,
>>>>>>>>>
>>>>>>>>> thanks a lot for reporting this issue. The problem arises whenever
>>>>>>>>> the old leader maintains the connection to ZooKeeper. If this is the case,
>>>>>>>>> then ephemeral nodes which we create to protect against faulty delete
>>>>>>>>> operations are not removed and consequently the new leader is not able to
>>>>>>>>> delete the persisted job graph. So one thing to check is whether the old JM
>>>>>>>>> still has an open connection to ZooKeeper. The next thing to check is the
>>>>>>>>> session timeout of your ZooKeeper cluster. If you stop the job within the
>>>>>>>>> session timeout, then it is also not guaranteed that ZooKeeper has detected
>>>>>>>>> that the ephemeral nodes of the old JM must be deleted. In order to
>>>>>>>>> understand this better it would be helpful if you could tell us the timing
>>>>>>>>> of the different actions.
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>> Till
>>>>>>>>>
>>>>>>>>> On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Encho,
>>>>>>>>>>
>>>>>>>>>> A temporary solution can be used to determine if it has been
>>>>>>>>>> cleaned up by monitoring the specific JobID under Zookeeper's "/jobgraph".
>>>>>>>>>> Another solution, modify the source code, rudely modify the
>>>>>>>>>> cleanup mode to the synchronous form, but the flink operation Zookeeper's
>>>>>>>>>> path needs to obtain the corresponding lock, so it is dangerous to do so,
>>>>>>>>>> and it is not recommended.
>>>>>>>>>> I think maybe this problem can be solved in the next version. It
>>>>>>>>>> depends on Till.
>>>>>>>>>>
>>>>>>>>>> Thanks, vino.
>>>>>>>>>>
>>>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:
>>>>>>>>>>
>>>>>>>>>>> Thank you very much for the info! Will keep track of the
>>>>>>>>>>> progress.
>>>>>>>>>>>
>>>>>>>>>>> In the meantime is there any viable workaround? It seems like HA
>>>>>>>>>>> doesn't really work due to this bug.
>>>>>>>>>>>
>>>>>>>>>>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> About some implementation mechanisms.
>>>>>>>>>>>> Flink uses Zookeeper to store JobGraph (Job's description
>>>>>>>>>>>> information and metadata) as a basis for Job recovery.
>>>>>>>>>>>> However, previous implementations may cause this information to
>>>>>>>>>>>> not be properly cleaned up because it is asynchronously deleted by a
>>>>>>>>>>>> background thread.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks, vino.
>>>>>>>>>>>>
>>>>>>>>>>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi Encho,
>>>>>>>>>>>>>
>>>>>>>>>>>>> This is a problem already known to the Flink community, you
>>>>>>>>>>>>> can track its progress through FLINK-10011[1], and currently Till is fixing
>>>>>>>>>>>>> this issue.
>>>>>>>>>>>>>
>>>>>>>>>>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks, vino.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一
>>>>>>>>>>>>> 下午10:13写道:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> I am running Flink 1.5.3 with two job managers and two task
>>>>>>>>>>>>>> managers in Kubernetes along with HDFS and Zookeeper in high-availability
>>>>>>>>>>>>>> mode.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> My problem occurs after the following actions:
>>>>>>>>>>>>>> - Upload a .jar file to jobmanager-1
>>>>>>>>>>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>>>>>>>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>>>>>>>>>>> - Kill pod of jobmanager-1
>>>>>>>>>>>>>> After a short delay, jobmanager-2 takes leadership and
>>>>>>>>>>>>>> correctly restores the job and continues it
>>>>>>>>>>>>>> - Stop job from jobmanager-2
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> At this point all seems well, but the problem is that
>>>>>>>>>>>>>> jobmanager-2 does not clean up anything that was left from jobmanager-1.
>>>>>>>>>>>>>> This means that both in HDFS and in Zookeeper remain job graphs, which
>>>>>>>>>>>>>> later on obstruct any work of both managers as after any reset they
>>>>>>>>>>>>>> unsuccessfully try to restore a non-existent job and fail over and over
>>>>>>>>>>>>>> again.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I am quite certain that jobmanager-2 does not know about any
>>>>>>>>>>>>>> of jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>>>>>>>>>>> duplicate job folders:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>>>>>>>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in
>>>>>>>>>>>>>> Zookeeper when the job is stopped, but fails since there are leftover files
>>>>>>>>>>>>>> in it from jobmanager-1:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>>>>>>>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I’ve noticed that when restoring the job, it seems like
>>>>>>>>>>>>>> jobmanager-2 does not get anything more than jobID, while it perhaps needs
>>>>>>>>>>>>>> some metadata? Here is the log that seems suspicious to me:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2018-08-27 13:09:18,113 INFO
>>>>>>>>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>>>>>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to
>>>>>>>>>>>>>> be aware that it’s overwriting anything or not deleting properly.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> My question is - what is the intended way for the job
>>>>>>>>>>>>>> managers to correctly exchange metadata in HA mode and why is it not
>>>>>>>>>>>>>> working for me?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks in advance!
>>>>>>>>>>>>>
>>>>>>>>>>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by Encho Mishinev <en...@gmail.com>.
Hi Till,

Those are actually the full logs except the two parts I shortened (pipeline
construction and execution). As I said - accessing the UI for Jobmanager 2
redirects to Jobmanager 1 so it seems like he is aware that he is not the
leader. Jobmanager 2 has no other logs than what I sent. Here is the full
end-to-end log of Jobmanager 2 after repeating the experiment again:

Starting Job Manager
sed: cannot rename /opt/flink/conf/sediVa6XS: Device or resource busy
config file:
jobmanager.rpc.address: flink-jobmanager-2
jobmanager.rpc.port: 6123
jobmanager.heap.size: 8192
taskmanager.heap.size: 8192
taskmanager.numberOfTaskSlots: 4
high-availability: zookeeper
high-availability.storageDir:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
high-availability.zookeeper.quorum: zk-cs:2181
high-availability.zookeeper.path.root: /flink
high-availability.jobmanager.port: 50010
state.backend: filesystem
state.checkpoints.dir:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
state.savepoints.dir:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
state.backend.incremental: false
fs.default-scheme:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
rest.port: 8081
web.upload.dir: /opt/flink/upload
query.server.port: 6125
taskmanager.numberOfTaskSlots: 4
classloader.parent-first-patterns.additional: org.apache.xerces.
blob.storage.directory: /opt/flink/blob-server
blob.server.port: 6124
blob.server.port: 6124
query.server.port: 6125
Starting standalonesession as a console application on host
flink-jobmanager-2-7844b78c9-zwdqv.
2018-08-29 13:19:24,047 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
--------------------------------------------------------------------------------
2018-08-29 13:19:24,049 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
Date:16.08.2018 @ 06:39:50 GMT)
2018-08-29 13:19:24,049 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
user: flink
2018-08-29 13:19:24,367 WARN  org.apache.hadoop.util.NativeCodeLoader
                 - Unable to load native-hadoop library for your
platform... using builtin-java classes where applicable
2018-08-29 13:19:24,431 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
Hadoop/Kerberos user: flink
2018-08-29 13:19:24,431 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
2018-08-29 13:19:24,431 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
heap size: 6702 MiBytes
2018-08-29 13:19:24,431 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
/docker-java-home/jre
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
version: 2.7.5
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
Options:
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
Arguments:
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 --configDir
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 /opt/flink/conf
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 --executionMode
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
2018-08-29 13:19:24,434 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
2018-08-29 13:19:24,435 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
2018-08-29 13:19:24,435 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
--------------------------------------------------------------------------------
2018-08-29 13:19:24,436 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
UNIX signal handlers for [TERM, HUP, INT]
2018-08-29 13:19:24,442 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: jobmanager.rpc.address, flink-jobmanager-2
2018-08-29 13:19:24,442 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: jobmanager.rpc.port, 6123
2018-08-29 13:19:24,442 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: jobmanager.heap.size, 8192
2018-08-29 13:19:24,442 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: taskmanager.heap.size, 8192
2018-08-29 13:19:24,442 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: taskmanager.numberOfTaskSlots, 4
2018-08-29 13:19:24,442 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability, zookeeper
2018-08-29 13:19:24,443 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.storageDir,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
2018-08-29 13:19:24,443 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.zookeeper.quorum, zk-cs:2181
2018-08-29 13:19:24,443 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.zookeeper.path.root, /flink
2018-08-29 13:19:24,443 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.jobmanager.port, 50010
2018-08-29 13:19:24,443 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.backend, filesystem
2018-08-29 13:19:24,443 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.checkpoints.dir,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
2018-08-29 13:19:24,444 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.savepoints.dir,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
2018-08-29 13:19:24,444 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.backend.incremental, false
2018-08-29 13:19:24,444 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: fs.default-scheme,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
2018-08-29 13:19:24,444 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: rest.port, 8081
2018-08-29 13:19:24,444 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: web.upload.dir, /opt/flink/upload
2018-08-29 13:19:24,444 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: query.server.port, 6125
2018-08-29 13:19:24,445 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: taskmanager.numberOfTaskSlots, 4
2018-08-29 13:19:24,445 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: classloader.parent-first-patterns.additional,
org.apache.xerces.
2018-08-29 13:19:24,445 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: blob.storage.directory, /opt/flink/blob-server
2018-08-29 13:19:24,445 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: blob.server.port, 6124
2018-08-29 13:19:24,445 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: blob.server.port, 6124
2018-08-29 13:19:24,445 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: query.server.port, 6125
2018-08-29 13:19:24,461 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
StandaloneSessionClusterEntrypoint.
2018-08-29 13:19:24,461 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
default filesystem.
2018-08-29 13:19:24,472 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
security context.
2018-08-29 13:19:24,506 INFO
org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
set to flink (auth:SIMPLE)
2018-08-29 13:19:24,522 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
Initializing cluster services.
2018-08-29 13:19:24,532 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
start actor system at flink-jobmanager-2:50010
2018-08-29 13:19:24,996 INFO  akka.event.slf4j.Slf4jLogger
                - Slf4jLogger started
2018-08-29 13:19:25,050 INFO  akka.remote.Remoting
                - Starting remoting
2018-08-29 13:19:25,209 INFO  akka.remote.Remoting
                - Remoting started; listening on addresses
:[akka.tcp://flink@flink-jobmanager-2:50010]
2018-08-29 13:19:25,216 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
system started at akka.tcp://flink@flink-jobmanager-2:50010
2018-08-29 13:19:25,648 INFO
org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
highly available BLOB storage directory at
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
2018-08-29 13:19:25,702 INFO  org.apache.flink.runtime.util.ZooKeeperUtils
                - Enforcing default ACL for ZK connections
2018-08-29 13:19:25,703 INFO  org.apache.flink.runtime.util.ZooKeeperUtils
                - Using '/flink/default' as Zookeeper namespace.
2018-08-29 13:19:25,750 INFO
org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
- Starting
2018-08-29 13:19:25,756 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
built on 03/23/2017 10:13 GMT
2018-08-29 13:19:25,756 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:host.name=flink-jobmanager-2-7844b78c9-zwdqv
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.version=1.8.0_181
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.vendor=Oracle Corporation
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.io.tmpdir=/tmp
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.compiler=<NA>
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:os.name=Linux
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:os.arch=amd64
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:os.version=4.4.0-1027-gke
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:user.name=flink
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:user.home=/opt/flink
2018-08-29 13:19:25,757 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:user.dir=/opt/flink
2018-08-29 13:19:25,758 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
2018-08-29 13:19:25,775 WARN
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
configuration failed: javax.security.auth.login.LoginException: No JAAS
configuration section named 'Client' was found in specified JAAS
configuration file: '/tmp/jaas-5000339768628554676.conf'. Will continue
connection to Zookeeper server without SASL authentication, if Zookeeper
server allows it.
2018-08-29 13:19:25,776 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
Opening socket connection to server zk-cs.default.svc.cluster.local/
10.27.248.104:2181
2018-08-29 13:19:25,777 ERROR
org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
Authentication failed
2018-08-29 13:19:25,777 INFO  org.apache.flink.runtime.blob.BlobServer
                - Created BLOB server storage directory
/opt/flink/blob-server/blobStore-40cefeee-e8d1-4522-aea3-957d9f7fbeee
2018-08-29 13:19:25,777 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
connection established to zk-cs.default.svc.cluster.local/10.27.248.104:2181,
initiating session
2018-08-29 13:19:25,778 INFO  org.apache.flink.runtime.blob.BlobServer
                - Started BLOB server at 0.0.0.0:6124 - max concurrent
requests: 50 - max backlog: 1000
2018-08-29 13:19:25,788 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
Session establishment complete on server zk-cs.default.svc.cluster.local/
10.27.248.104:2181, sessionid = 0x26584fd55690009, negotiated timeout =
40000
2018-08-29 13:19:25,789 INFO
org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
- State change: CONNECTED
2018-08-29 13:19:25,793 INFO
org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
reporter configured, no metrics will be exposed/reported.
2018-08-29 13:19:25,798 INFO
org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
Initializing FileArchivedExecutionGraphStore: Storage directory
/tmp/executionGraphStore-76cce4e7-84ea-4624-a847-bbd7fdc4f109, expiration
time 3600000, maximum cache size 52428800 bytes.
2018-08-29 13:19:25,824 INFO
org.apache.flink.runtime.blob.TransientBlobCache              - Created
BLOB cache storage directory
/opt/flink/blob-server/blobStore-7c6c2db0-f7ab-4cb6-909d-6c9cbfd78215
2018-08-29 13:19:25,838 WARN  org.apache.flink.configuration.Configuration
                - Config uses deprecated configuration key
'jobmanager.rpc.address' instead of proper key 'rest.address'
2018-08-29 13:19:25,839 WARN
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
directory /opt/flink/upload/flink-web-upload does not exist, or has been
deleted externally. Previously uploaded files are no longer available.
2018-08-29 13:19:25,840 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
directory /opt/flink/upload/flink-web-upload for file uploads.
2018-08-29 13:19:25,843 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
rest endpoint.
2018-08-29 13:19:26,143 WARN
org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
environment variable 'log.file' is not set.
2018-08-29 13:19:26,143 WARN
org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
log files are unavailable in the web dashboard. Log file location not found
in environment variable 'log.file' or configuration key 'Key:
'web.log.path' , default: null (deprecated keys:
[jobmanager.web.log.path])'.
2018-08-29 13:19:26,216 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
endpoint listening at flink-jobmanager-2:8081
2018-08-29 13:19:26,217 INFO
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
Starting ZooKeeperLeaderElectionService
ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
2018-08-29 13:19:26,236 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
frontend listening at http://flink-jobmanager-2:8081.
2018-08-29 13:19:26,248 INFO
org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
RPC endpoint for
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
akka://flink/user/resourcemanager .
2018-08-29 13:19:26,323 INFO
org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
at akka://flink/user/dispatcher .
2018-08-29 13:19:26,335 INFO
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
Starting ZooKeeperLeaderElectionService
ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
2018-08-29 13:19:26,336 INFO
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2018-08-29 13:19:26,338 INFO
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
Starting ZooKeeperLeaderElectionService
ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
2018-08-29 13:19:26,339 INFO
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
2018-08-29 13:23:21,513 INFO
org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
Recovered SubmittedJobGraph(836b29f7c66bbeb6ed8bae41cb9b316c, null).

Thanks,
Encho

On Wed, Aug 29, 2018 at 3:59 PM Till Rohrmann <tr...@apache.org> wrote:

> Hi Encho,
>
> thanks for sending the first part of the logs. What I would actually be
> interested in are the complete logs because somewhere in the jobmanager-2
> logs there must be a log statement saying that the respective dispatcher
> gained leadership. I would like to see why this happens but for this to
> debug the complete logs are necessary. It would be awesome if you could
> send them to me. Thanks a lot!
>
> Cheers,
> Till
>
> On Wed, Aug 29, 2018 at 2:00 PM Encho Mishinev <en...@gmail.com>
> wrote:
>
>> Hi Till,
>>
>> I will use the approach with a k8s deployment and HA mode with a single
>> job manager. Nonetheless, here are the logs I just produced by repeating
>> the aforementioned experiment, hope they help in debugging:
>>
>> *- Starting Jobmanager-1:*
>>
>> Starting Job Manager
>> sed: cannot rename /opt/flink/conf/sedR98XPn: Device or resource busy
>> config file:
>> jobmanager.rpc.address: flink-jobmanager-1
>> jobmanager.rpc.port: 6123
>> jobmanager.heap.size: 8192
>> taskmanager.heap.size: 8192
>> taskmanager.numberOfTaskSlots: 4
>> high-availability: zookeeper
>> high-availability.storageDir:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>> high-availability.zookeeper.quorum: zk-cs:2181
>> high-availability.zookeeper.path.root: /flink
>> high-availability.jobmanager.port: 50010
>> state.backend: filesystem
>> state.checkpoints.dir:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>> state.savepoints.dir:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>> state.backend.incremental: false
>> fs.default-scheme:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>> rest.port: 8081
>> web.upload.dir: /opt/flink/upload
>> query.server.port: 6125
>> taskmanager.numberOfTaskSlots: 4
>> classloader.parent-first-patterns.additional: org.apache.xerces.
>> blob.storage.directory: /opt/flink/blob-server
>> blob.server.port: 6124
>> blob.server.port: 6124
>> query.server.port: 6125
>> Starting standalonesession as a console application on host
>> flink-jobmanager-1-f76fd4df8-ftwt9.
>> 2018-08-29 11:41:48,806 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>> --------------------------------------------------------------------------------
>> 2018-08-29 11:41:48,807 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
>> StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
>> Date:16.08.2018 @ 06:39:50 GMT)
>> 2018-08-29 11:41:48,807 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
>> user: flink
>> 2018-08-29 11:41:49,134 WARN  org.apache.hadoop.util.NativeCodeLoader
>>                    - Unable to load native-hadoop library for your
>> platform... using builtin-java classes where applicable
>> 2018-08-29 11:41:49,210 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
>> Hadoop/Kerberos user: flink
>> 2018-08-29 11:41:49,210 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
>> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
>> 2018-08-29 11:41:49,210 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
>> heap size: 6702 MiBytes
>> 2018-08-29 11:41:49,210 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
>> /docker-java-home/jre
>> 2018-08-29 11:41:49,213 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
>> version: 2.7.5
>> 2018-08-29 11:41:49,213 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
>> Options:
>> 2018-08-29 11:41:49,213 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
>> 2018-08-29 11:41:49,213 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
>> 2018-08-29 11:41:49,213 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
>> Arguments:
>> 2018-08-29 11:41:49,213 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  --configDir
>> 2018-08-29 11:41:49,213 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  /opt/flink/conf
>> 2018-08-29 11:41:49,213 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  --executionMode
>> 2018-08-29 11:41:49,213 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>> 2018-08-29 11:41:49,214 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
>> 2018-08-29 11:41:49,214 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>> 2018-08-29 11:41:49,214 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
>> /opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>> 2018-08-29 11:41:49,214 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>> --------------------------------------------------------------------------------
>> 2018-08-29 11:41:49,215 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
>> UNIX signal handlers for [TERM, HUP, INT]
>> 2018-08-29 11:41:49,221 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: jobmanager.rpc.address, flink-jobmanager-1
>> 2018-08-29 11:41:49,221 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: jobmanager.rpc.port, 6123
>> 2018-08-29 11:41:49,221 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: jobmanager.heap.size, 8192
>> 2018-08-29 11:41:49,221 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: taskmanager.heap.size, 8192
>> 2018-08-29 11:41:49,221 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: taskmanager.numberOfTaskSlots, 4
>> 2018-08-29 11:41:49,222 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability, zookeeper
>> 2018-08-29 11:41:49,222 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.storageDir,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>> 2018-08-29 11:41:49,222 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.zookeeper.quorum, zk-cs:2181
>> 2018-08-29 11:41:49,222 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.zookeeper.path.root, /flink
>> 2018-08-29 11:41:49,223 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.jobmanager.port, 50010
>> 2018-08-29 11:41:49,223 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.backend, filesystem
>> 2018-08-29 11:41:49,223 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.checkpoints.dir,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>> 2018-08-29 11:41:49,223 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.savepoints.dir,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>> 2018-08-29 11:41:49,223 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.backend.incremental, false
>> 2018-08-29 11:41:49,224 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: fs.default-scheme,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>> 2018-08-29 11:41:49,224 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: rest.port, 8081
>> 2018-08-29 11:41:49,224 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: web.upload.dir, /opt/flink/upload
>> 2018-08-29 11:41:49,224 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: query.server.port, 6125
>> 2018-08-29 11:41:49,225 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: taskmanager.numberOfTaskSlots, 4
>> 2018-08-29 11:41:49,225 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: classloader.parent-first-patterns.additional,
>> org.apache.xerces.
>> 2018-08-29 11:41:49,225 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: blob.storage.directory, /opt/flink/blob-server
>> 2018-08-29 11:41:49,225 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: blob.server.port, 6124
>> 2018-08-29 11:41:49,225 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: blob.server.port, 6124
>> 2018-08-29 11:41:49,225 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: query.server.port, 6125
>> 2018-08-29 11:41:49,239 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
>> StandaloneSessionClusterEntrypoint.
>> 2018-08-29 11:41:49,239 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>> default filesystem.
>> 2018-08-29 11:41:49,250 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>> security context.
>> 2018-08-29 11:41:49,282 INFO
>> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
>> set to flink (auth:SIMPLE)
>> 2018-08-29 11:41:49,298 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>> Initializing cluster services.
>> 2018-08-29 11:41:49,309 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
>> start actor system at flink-jobmanager-1:50010
>> 2018-08-29 11:41:49,768 INFO  akka.event.slf4j.Slf4jLogger
>>                   - Slf4jLogger started
>> 2018-08-29 11:41:49,823 INFO  akka.remote.Remoting
>>                   - Starting remoting
>> 2018-08-29 11:41:49,974 INFO  akka.remote.Remoting
>>                   - Remoting started; listening on addresses
>> :[akka.tcp://flink@flink-jobmanager-1:50010]
>> 2018-08-29 11:41:49,981 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
>> system started at akka.tcp://flink@flink-jobmanager-1:50010
>> 2018-08-29 11:41:50,444 INFO
>> org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
>> highly available BLOB storage directory at
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
>> 2018-08-29 11:41:50,509 INFO
>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing
>> default ACL for ZK connections
>> 2018-08-29 11:41:50,509 INFO
>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Using
>> '/flink/default' as Zookeeper namespace.
>> 2018-08-29 11:41:50,568 INFO
>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
>> - Starting
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
>> built on 03/23/2017 10:13 GMT
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:host.name=flink-jobmanager-1-f76fd4df8-ftwt9
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.version=1.8.0_181
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.vendor=Oracle Corporation
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.io.tmpdir=/tmp
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.compiler=<NA>
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:os.name=Linux
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:os.arch=amd64
>> 2018-08-29 11:41:50,577 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:os.version=4.4.0-1027-gke
>> 2018-08-29 11:41:50,578 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:user.name=flink
>> 2018-08-29 11:41:50,578 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:user.home=/opt/flink
>> 2018-08-29 11:41:50,578 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:user.dir=/opt/flink
>> 2018-08-29 11:41:50,578 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
>> Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
>> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
>> 2018-08-29 11:41:50,605 INFO  org.apache.flink.runtime.blob.BlobServer
>>                   - Created BLOB server storage directory
>> /opt/flink/blob-server/blobStore-d408cea8-2ed0-461a-a30a-a62b70fd332a
>> 2018-08-29 11:41:50,605 WARN
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
>> configuration failed: javax.security.auth.login.LoginException: No JAAS
>> configuration section named 'Client' was found in specified JAAS
>> configuration file: '/tmp/jaas-5372401662150571998.conf'. Will continue
>> connection to Zookeeper server without SASL authentication, if Zookeeper
>> server allows it.
>> 2018-08-29 11:41:50,607 INFO  org.apache.flink.runtime.blob.BlobServer
>>                   - Started BLOB server at 0.0.0.0:6124 - max concurrent
>> requests: 50 - max backlog: 1000
>> 2018-08-29 11:41:50,607 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>> Opening socket connection to server zk-cs.default.svc.cluster.local/
>> 10.27.248.104:2181
>> 2018-08-29 11:41:50,608 ERROR
>> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
>> Authentication failed
>> 2018-08-29 11:41:50,609 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
>> connection established to zk-cs.default.svc.cluster.local/
>> 10.27.248.104:2181, initiating session
>> 2018-08-29 11:41:50,618 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>> Session establishment complete on server zk-cs.default.svc.cluster.local/
>> 10.27.248.104:2181, sessionid = 0x26584fd55690005, negotiated timeout =
>> 40000
>> 2018-08-29 11:41:50,619 INFO
>> org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
>> - State change: CONNECTED
>> 2018-08-29 11:41:50,627 INFO
>> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
>> reporter configured, no metrics will be exposed/reported.
>> 2018-08-29 11:41:50,633 INFO
>> org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
>> Initializing FileArchivedExecutionGraphStore: Storage directory
>> /tmp/executionGraphStore-c5df0b39-86f3-4fba-bdda-aacca4f86086, expiration
>> time 3600000, maximum cache size 52428800 bytes.
>> 2018-08-29 11:41:50,659 INFO
>> org.apache.flink.runtime.blob.TransientBlobCache              - Created
>> BLOB cache storage directory
>> /opt/flink/blob-server/blobStore-c12d55af-3c2d-4fc2-8ee8-6de642522184
>> 2018-08-29 11:41:50,674 WARN
>> org.apache.flink.configuration.Configuration                  - Config uses
>> deprecated configuration key 'jobmanager.rpc.address' instead of proper key
>> 'rest.address'
>> 2018-08-29 11:41:50,675 WARN
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
>> directory /opt/flink/upload/flink-web-upload does not exist, or has been
>> deleted externally. Previously uploaded files are no longer available.
>> 2018-08-29 11:41:50,676 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
>> directory /opt/flink/upload/flink-web-upload for file uploads.
>> 2018-08-29 11:41:50,679 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
>> rest endpoint.
>> 2018-08-29 11:41:50,995 WARN
>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
>> environment variable 'log.file' is not set.
>> 2018-08-29 11:41:50,995 WARN
>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
>> log files are unavailable in the web dashboard. Log file location not found
>> in environment variable 'log.file' or configuration key 'Key:
>> 'web.log.path' , default: null (deprecated keys:
>> [jobmanager.web.log.path])'.
>> 2018-08-29 11:41:51,071 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
>> endpoint listening at flink-jobmanager-1:8081
>> 2018-08-29 11:41:51,071 INFO
>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>> Starting ZooKeeperLeaderElectionService
>> ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
>> 2018-08-29 11:41:51,091 WARN
>> org.apache.flink.shaded.curator.org.apache.curator.utils.ZKPaths  - The
>> version of ZooKeeper being used doesn't support Container nodes.
>> CreateMode.PERSISTENT will be used instead.
>> 2018-08-29 11:41:51,101 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
>> frontend listening at http://flink-jobmanager-1:8081.
>> 2018-08-29 11:41:51,114 INFO
>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>> RPC endpoint for
>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
>> akka://flink/user/resourcemanager .
>> 2018-08-29 11:41:51,141 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    -
>> http://flink-jobmanager-1:8081 was granted leadership with
>> leaderSessionID=bb0d4dfd-c2c4-480b-bc86-62e231a606dd
>> 2018-08-29 11:41:51,214 INFO
>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>> RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
>> at akka://flink/user/dispatcher .
>> 2018-08-29 11:41:51,230 INFO
>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>> Starting ZooKeeperLeaderElectionService
>> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
>> 2018-08-29 11:41:51,232 INFO
>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>> Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>> 2018-08-29 11:41:51,234 INFO
>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>> Starting ZooKeeperLeaderElectionService
>> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
>> 2018-08-29 11:41:51,235 INFO
>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>> 2018-08-29 11:41:51,253 INFO
>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  -
>> ResourceManager akka.tcp://flink@flink-jobmanager-1:50010/user/resourcemanager
>> was granted leadership with fencing token ba47ed8daa8ff16bea6fc355c13f4d49
>> 2018-08-29 11:41:51,254 INFO
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
>> Starting the SlotManager.
>> 2018-08-29 11:41:51,263 INFO
>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Dispatcher
>> akka.tcp://flink@flink-jobmanager-1:50010/user/dispatcher was granted
>> leadership with fencing token 703301bf-85e7-4464-990f-ad39128a7b4d
>> 2018-08-29 11:41:51,263 INFO
>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Recovering
>> all persisted jobs.
>> 2018-08-29 11:41:51,468 INFO
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
>> Registering TaskManager c8a3201d58d87dbbe16f8eb352b5c5b6 under
>> 1c5bf0bc3848bd384b6f032ff7213754 at the SlotManager.
>> 2018-08-29 11:41:51,471 INFO
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
>> Registering TaskManager 104d18b72fed054620e58e120a1ea083 under
>> e9d3e8ad3b477dd2e58bcb88a2c0d061 at the SlotManager.
>>
>> *Starting Jobmanager-2:*
>>
>> Starting Job Manager
>> sed: cannot rename /opt/flink/conf/sedH2ZiSu: Device or resource busy
>> config file:
>> jobmanager.rpc.address: flink-jobmanager-2
>> jobmanager.rpc.port: 6123
>> jobmanager.heap.size: 8192
>> taskmanager.heap.size: 8192
>> taskmanager.numberOfTaskSlots: 4
>> high-availability: zookeeper
>> high-availability.storageDir:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>> high-availability.zookeeper.quorum: zk-cs:2181
>> high-availability.zookeeper.path.root: /flink
>> high-availability.jobmanager.port: 50010
>> state.backend: filesystem
>> state.checkpoints.dir:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>> state.savepoints.dir:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>> state.backend.incremental: false
>> fs.default-scheme:
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>> rest.port: 8081
>> web.upload.dir: /opt/flink/upload
>> query.server.port: 6125
>> taskmanager.numberOfTaskSlots: 4
>> classloader.parent-first-patterns.additional: org.apache.xerces.
>> blob.storage.directory: /opt/flink/blob-server
>> blob.server.port: 6124
>> blob.server.port: 6124
>> query.server.port: 6125
>> Starting standalonesession as a console application on host
>> flink-jobmanager-2-7844b78c9-kmvw9.
>> 2018-08-29 11:41:51,688 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>> --------------------------------------------------------------------------------
>> 2018-08-29 11:41:51,690 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
>> StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
>> Date:16.08.2018 @ 06:39:50 GMT)
>> 2018-08-29 11:41:51,690 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
>> user: flink
>> 2018-08-29 11:41:52,018 WARN  org.apache.hadoop.util.NativeCodeLoader
>>                    - Unable to load native-hadoop library for your
>> platform... using builtin-java classes where applicable
>> 2018-08-29 11:41:52,088 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
>> Hadoop/Kerberos user: flink
>> 2018-08-29 11:41:52,088 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
>> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
>> 2018-08-29 11:41:52,088 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
>> heap size: 6702 MiBytes
>> 2018-08-29 11:41:52,088 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
>> /docker-java-home/jre
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
>> version: 2.7.5
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
>> Options:
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
>> Arguments:
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  --configDir
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  /opt/flink/conf
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>  --executionMode
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
>> /opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>> 2018-08-29 11:41:52,091 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>> --------------------------------------------------------------------------------
>> 2018-08-29 11:41:52,092 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
>> UNIX signal handlers for [TERM, HUP, INT]
>> 2018-08-29 11:41:52,103 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: jobmanager.rpc.address, flink-jobmanager-2
>> 2018-08-29 11:41:52,103 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: jobmanager.rpc.port, 6123
>> 2018-08-29 11:41:52,103 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: jobmanager.heap.size, 8192
>> 2018-08-29 11:41:52,104 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: taskmanager.heap.size, 8192
>> 2018-08-29 11:41:52,104 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: taskmanager.numberOfTaskSlots, 4
>> 2018-08-29 11:41:52,104 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability, zookeeper
>> 2018-08-29 11:41:52,104 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.storageDir,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
>> 2018-08-29 11:41:52,104 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.zookeeper.quorum, zk-cs:2181
>> 2018-08-29 11:41:52,104 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.zookeeper.path.root, /flink
>> 2018-08-29 11:41:52,105 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: high-availability.jobmanager.port, 50010
>> 2018-08-29 11:41:52,105 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.backend, filesystem
>> 2018-08-29 11:41:52,105 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.checkpoints.dir,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
>> 2018-08-29 11:41:52,105 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.savepoints.dir,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
>> 2018-08-29 11:41:52,105 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: state.backend.incremental, false
>> 2018-08-29 11:41:52,106 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: fs.default-scheme,
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
>> 2018-08-29 11:41:52,106 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: rest.port, 8081
>> 2018-08-29 11:41:52,106 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: web.upload.dir, /opt/flink/upload
>> 2018-08-29 11:41:52,106 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: query.server.port, 6125
>> 2018-08-29 11:41:52,106 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: taskmanager.numberOfTaskSlots, 4
>> 2018-08-29 11:41:52,107 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: classloader.parent-first-patterns.additional,
>> org.apache.xerces.
>> 2018-08-29 11:41:52,107 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: blob.storage.directory, /opt/flink/blob-server
>> 2018-08-29 11:41:52,107 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: blob.server.port, 6124
>> 2018-08-29 11:41:52,107 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: blob.server.port, 6124
>> 2018-08-29 11:41:52,107 INFO
>> org.apache.flink.configuration.GlobalConfiguration            - Loading
>> configuration property: query.server.port, 6125
>> 2018-08-29 11:41:52,122 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
>> StandaloneSessionClusterEntrypoint.
>> 2018-08-29 11:41:52,123 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>> default filesystem.
>> 2018-08-29 11:41:52,133 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
>> security context.
>> 2018-08-29 11:41:52,173 INFO
>> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
>> set to flink (auth:SIMPLE)
>> 2018-08-29 11:41:52,188 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>> Initializing cluster services.
>> 2018-08-29 11:41:52,198 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
>> start actor system at flink-jobmanager-2:50010
>> 2018-08-29 11:41:52,753 INFO  akka.event.slf4j.Slf4jLogger
>>                   - Slf4jLogger started
>> 2018-08-29 11:41:52,822 INFO  akka.remote.Remoting
>>                   - Starting remoting
>> 2018-08-29 11:41:53,038 INFO  akka.remote.Remoting
>>                   - Remoting started; listening on addresses
>> :[akka.tcp://flink@flink-jobmanager-2:50010]
>> 2018-08-29 11:41:53,046 INFO
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
>> system started at akka.tcp://flink@flink-jobmanager-2:50010
>> 2018-08-29 11:41:53,500 INFO
>> org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
>> highly available BLOB storage directory at
>> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
>> 2018-08-29 11:41:53,558 INFO
>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing
>> default ACL for ZK connections
>> 2018-08-29 11:41:53,559 INFO
>> org.apache.flink.runtime.util.ZooKeeperUtils                  - Using
>> '/flink/default' as Zookeeper namespace.
>> 2018-08-29 11:41:53,616 INFO
>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
>> - Starting
>> 2018-08-29 11:41:53,624 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
>> built on 03/23/2017 10:13 GMT
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:host.name=flink-jobmanager-2-7844b78c9-kmvw9
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.version=1.8.0_181
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.vendor=Oracle Corporation
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.io.tmpdir=/tmp
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:java.compiler=<NA>
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:os.name=Linux
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:os.arch=amd64
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:os.version=4.4.0-1027-gke
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:user.name=flink
>> 2018-08-29 11:41:53,625 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:user.home=/opt/flink
>> 2018-08-29 11:41:53,626 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>> environment:user.dir=/opt/flink
>> 2018-08-29 11:41:53,626 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
>> Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
>> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
>> 2018-08-29 11:41:53,644 WARN
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
>> configuration failed: javax.security.auth.login.LoginException: No JAAS
>> configuration section named 'Client' was found in specified JAAS
>> configuration file: '/tmp/jaas-8238466329925822361.conf'. Will continue
>> connection to Zookeeper server without SASL authentication, if Zookeeper
>> server allows it.
>> 2018-08-29 11:41:53,646 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>> Opening socket connection to server zk-cs.default.svc.cluster.local/
>> 10.27.248.104:2181
>> 2018-08-29 11:41:53,646 INFO  org.apache.flink.runtime.blob.BlobServer
>>                   - Created BLOB server storage directory
>> /opt/flink/blob-server/blobStore-61cdb645-5d0c-47fd-bcf6-84ad16fadade
>> 2018-08-29 11:41:53,646 ERROR
>> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
>> Authentication failed
>> 2018-08-29 11:41:53,647 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
>> connection established to zk-cs.default.svc.cluster.local/
>> 10.27.248.104:2181, initiating session
>> 2018-08-29 11:41:53,649 INFO  org.apache.flink.runtime.blob.BlobServer
>>                   - Started BLOB server at 0.0.0.0:6124 - max concurrent
>> requests: 50 - max backlog: 1000
>> 2018-08-29 11:41:53,655 INFO
>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
>> Session establishment complete on server zk-cs.default.svc.cluster.local/
>> 10.27.248.104:2181, sessionid = 0x26584fd55690006, negotiated timeout =
>> 40000
>> 2018-08-29 11:41:53,656 INFO
>> org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
>> - State change: CONNECTED
>> 2018-08-29 11:41:53,667 INFO
>> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
>> reporter configured, no metrics will be exposed/reported.
>> 2018-08-29 11:41:53,673 INFO
>> org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
>> Initializing FileArchivedExecutionGraphStore: Storage directory
>> /tmp/executionGraphStore-8b236c14-79ee-4a84-b23f-437408c4661a, expiration
>> time 3600000, maximum cache size 52428800 bytes.
>> 2018-08-29 11:41:53,699 INFO
>> org.apache.flink.runtime.blob.TransientBlobCache              - Created
>> BLOB cache storage directory
>> /opt/flink/blob-server/blobStore-80c519df-cc6f-4e9c-9cd5-da4077c826f0
>> 2018-08-29 11:41:53,717 WARN
>> org.apache.flink.configuration.Configuration                  - Config uses
>> deprecated configuration key 'jobmanager.rpc.address' instead of proper key
>> 'rest.address'
>> 2018-08-29 11:41:53,718 WARN
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
>> directory /opt/flink/upload/flink-web-upload does not exist, or has been
>> deleted externally. Previously uploaded files are no longer available.
>> 2018-08-29 11:41:53,719 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
>> directory /opt/flink/upload/flink-web-upload for file uploads.
>> 2018-08-29 11:41:53,722 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
>> rest endpoint.
>> 2018-08-29 11:41:54,084 WARN
>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
>> environment variable 'log.file' is not set.
>> 2018-08-29 11:41:54,084 WARN
>> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
>> log files are unavailable in the web dashboard. Log file location not found
>> in environment variable 'log.file' or configuration key 'Key:
>> 'web.log.path' , default: null (deprecated keys:
>> [jobmanager.web.log.path])'.
>> 2018-08-29 11:41:54,160 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
>> endpoint listening at flink-jobmanager-2:8081
>> 2018-08-29 11:41:54,160 INFO
>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>> Starting ZooKeeperLeaderElectionService
>> ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
>> 2018-08-29 11:41:54,180 INFO
>> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
>> frontend listening at http://flink-jobmanager-2:8081.
>> 2018-08-29 11:41:54,192 INFO
>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>> RPC endpoint for
>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
>> akka://flink/user/resourcemanager .
>> 2018-08-29 11:41:54,273 INFO
>> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
>> RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
>> at akka://flink/user/dispatcher .
>> 2018-08-29 11:41:54,286 INFO
>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>> Starting ZooKeeperLeaderElectionService
>> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
>> 2018-08-29 11:41:54,287 INFO
>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>> Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>> 2018-08-29 11:41:54,289 INFO
>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>> Starting ZooKeeperLeaderElectionService
>> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
>> 2018-08-29 11:41:54,289 INFO
>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>
>> *Upon submitting a batch job on Jobmanager-1, we immediately get this log
>> on Jobmanager-2*
>> 2018-08-29 11:47:06,249 INFO
>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>> Recovered SubmittedJobGraph(d69b67e4d28a2d244b06d3f6d661bca1, null).
>>
>> *Meanwhile Jobmanager-1 gets:*
>> *-FlinkBatchPipelineTranslator pipeline logs- (we use Apache Beam)*
>>
>> 2018-08-29 11:47:06,006 INFO
>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Submitting
>> job d69b67e4d28a2d244b06d3f6d661bca1
>> (sicassandrawriterbeam-flink-0829114703-7d95fabd).
>> 2018-08-29 11:47:06,090 INFO
>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>> Added SubmittedJobGraph(d69b67e4d28a2d244b06d3f6d661bca1, null) to
>> ZooKeeper.
>>
>> *-loads of job execution info-*
>>
>> 2018-08-29 11:49:20,272 INFO
>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Job
>> d69b67e4d28a2d244b06d3f6d661bca1 reached globally terminal state FINISHED.
>> 2018-08-29 11:49:20,286 INFO
>> org.apache.flink.runtime.jobmaster.JobMaster                  - Stopping
>> the JobMaster for job
>> sicassandrawriterbeam-flink-0829114703-7d95fabd(d69b67e4d28a2d244b06d3f6d661bca1).
>> 2018-08-29 11:49:20,290 INFO
>> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>> Stopping ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>> 2018-08-29 11:49:20,292 INFO
>> org.apache.flink.runtime.jobmaster.JobMaster                  - Close
>> ResourceManager connection 827b94881bf7c94d8516907e04e3a564: JobManager is
>> shutting down..
>> 2018-08-29 11:49:20,292 INFO
>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Suspending
>> SlotPool.
>> 2018-08-29 11:49:20,293 INFO
>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Stopping
>> SlotPool.
>> 2018-08-29 11:49:20,293 INFO
>> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  -
>> Disconnect job manager a3dab0a0883c5f0f37943358d9104d79
>> @akka.tcp://flink@flink-jobmanager-1:50010/user/jobmanager_0 for job
>> d69b67e4d28a2d244b06d3f6d661bca1 from the resource manager.
>> 2018-08-29 11:49:20,293 INFO
>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>> Stopping ZooKeeperLeaderElectionService
>> ZooKeeperLeaderElectionService{leaderPath='/leader/d69b67e4d28a2d244b06d3f6d661bca1/job_manager_lock'}.
>> 2018-08-29 11:49:20,304 INFO
>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>> Removed job graph d69b67e4d28a2d244b06d3f6d661bca1 from ZooKeeper.
>>
>>
>> -------------------
>>
>> The result is:
>> HDFS has only a jobgraph and an empty default folder - everything else is
>> cleared
>> ZooKeeper has the jobgraph that Jobmanager-1 claims to have removed in
>> the last log still there.
>>
>> On Wed, Aug 29, 2018 at 12:14 PM Till Rohrmann <tr...@apache.org>
>> wrote:
>>
>>> Hi Encho,
>>>
>>> it sounds strange that the standby JobManager tries to recover a
>>> submitted job graph. This should only happen if it has been granted
>>> leadership. Thus, it seems as if the standby JobManager thinks that it is
>>> also the leader. Could you maybe share the logs of the two
>>> JobManagers/ClusterEntrypoints with us?
>>>
>>> Running only a single JobManager/ClusterEntrypoint in HA mode via a
>>> Kubernetes Deployment should do the trick and there is nothing wrong with
>>> it.
>>>
>>> Cheers,
>>> Till
>>>
>>> On Wed, Aug 29, 2018 at 11:05 AM Encho Mishinev <
>>> encho.mishinev@gmail.com> wrote:
>>>
>>>> Hello,
>>>>
>>>> Since two job managers don't seem to be working for me I was thinking
>>>> of just using a single job manager in Kubernetes in HA mode with a
>>>> deployment ensuring its restart whenever it fails. Is this approach viable?
>>>> The High-Availability page mentions that you use only one job manager in an
>>>> YARN cluster but does not specify such option for Kubernetes. Is there
>>>> anything that can go wrong with this approach?
>>>>
>>>> Thanks
>>>>
>>>> On Wed, Aug 29, 2018 at 11:10 AM Encho Mishinev <
>>>> encho.mishinev@gmail.com> wrote:
>>>>
>>>>> Hi,
>>>>>
>>>>> Unfortunately the thing I described does indeed happen every time. As
>>>>> mentioned in the first email, I am running on Kubernetes so certain things
>>>>> could be different compared to just a standalone cluster.
>>>>>
>>>>> Any ideas for workarounds are welcome, as this problem basically
>>>>> prevents me from using HA.
>>>>>
>>>>> Thanks,
>>>>> Encho
>>>>>
>>>>> On Wed, Aug 29, 2018 at 5:15 AM vino yang <ya...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi Encho,
>>>>>>
>>>>>> From your description, I feel that there are extra bugs.
>>>>>>
>>>>>> About your description:
>>>>>>
>>>>>> *- Start both job managers*
>>>>>> *- Start a batch job in JobManager 1 and let it finish*
>>>>>> *The jobgraphs in both Zookeeper and HDFS remained.*
>>>>>>
>>>>>> Is it necessarily happening every time?
>>>>>>
>>>>>> In the Standalone cluster, the problems we encountered were sporadic.
>>>>>>
>>>>>> Thanks, vino.
>>>>>>
>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午8:07写道:
>>>>>>
>>>>>>> Hello Till,
>>>>>>>
>>>>>>> I spend a few more hours testing and looking at the logs and it
>>>>>>> seems like there's a more general problem here. While the two job managers
>>>>>>> are active neither of them can properly delete jobgraphs. The above problem
>>>>>>> I described comes from the fact that Kubernetes gets JobManager 1 quickly
>>>>>>> after I manually kill it, so when I stop the job on JobManager 2 both are
>>>>>>> alive.
>>>>>>>
>>>>>>> I did a very simple test:
>>>>>>>
>>>>>>> - Start both job managers
>>>>>>> - Start a batch job in JobManager 1 and let it finish
>>>>>>> The jobgraphs in both Zookeeper and HDFS remained.
>>>>>>>
>>>>>>> On the other hand if we do:
>>>>>>>
>>>>>>> - Start only JobManager 1 (again in HA mode)
>>>>>>> - Start a batch job and let it finish
>>>>>>> The jobgraphs in both Zookeeper and HDFS are deleted fine.
>>>>>>>
>>>>>>> It seems like the standby manager still leaves some kind of lock on
>>>>>>> the jobgraphs. Do you think that's possible? Have you seen a similar
>>>>>>> problem?
>>>>>>> The only logs that appear on the standby manager while waiting are
>>>>>>> of the type:
>>>>>>>
>>>>>>> 2018-08-28 11:54:10,789 INFO
>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>> Recovered SubmittedJobGraph(9e0a109b57511930c95d3b54574a66e3, null).
>>>>>>>
>>>>>>> Note that this log appears on the standby jobmanager immediately
>>>>>>> when a new job is submitted to the active jobmanager.
>>>>>>> Also note that the blobs and checkpoints are cleared fine. The
>>>>>>> problem is only for jobgraphs both in ZooKeeper and HDFS.
>>>>>>>
>>>>>>> Trying to access the UI of the standby manager redirects to the
>>>>>>> active one, so it is not a problem of them not knowing who the leader is.
>>>>>>> Do you have any ideas?
>>>>>>>
>>>>>>> Thanks a lot,
>>>>>>> Encho
>>>>>>>
>>>>>>> On Tue, Aug 28, 2018 at 10:27 AM Till Rohrmann <tr...@apache.org>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Encho,
>>>>>>>>
>>>>>>>> thanks a lot for reporting this issue. The problem arises whenever
>>>>>>>> the old leader maintains the connection to ZooKeeper. If this is the case,
>>>>>>>> then ephemeral nodes which we create to protect against faulty delete
>>>>>>>> operations are not removed and consequently the new leader is not able to
>>>>>>>> delete the persisted job graph. So one thing to check is whether the old JM
>>>>>>>> still has an open connection to ZooKeeper. The next thing to check is the
>>>>>>>> session timeout of your ZooKeeper cluster. If you stop the job within the
>>>>>>>> session timeout, then it is also not guaranteed that ZooKeeper has detected
>>>>>>>> that the ephemeral nodes of the old JM must be deleted. In order to
>>>>>>>> understand this better it would be helpful if you could tell us the timing
>>>>>>>> of the different actions.
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Till
>>>>>>>>
>>>>>>>> On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Encho,
>>>>>>>>>
>>>>>>>>> A temporary solution can be used to determine if it has been
>>>>>>>>> cleaned up by monitoring the specific JobID under Zookeeper's "/jobgraph".
>>>>>>>>> Another solution, modify the source code, rudely modify the
>>>>>>>>> cleanup mode to the synchronous form, but the flink operation Zookeeper's
>>>>>>>>> path needs to obtain the corresponding lock, so it is dangerous to do so,
>>>>>>>>> and it is not recommended.
>>>>>>>>> I think maybe this problem can be solved in the next version. It
>>>>>>>>> depends on Till.
>>>>>>>>>
>>>>>>>>> Thanks, vino.
>>>>>>>>>
>>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:
>>>>>>>>>
>>>>>>>>>> Thank you very much for the info! Will keep track of the
>>>>>>>>>> progress.
>>>>>>>>>>
>>>>>>>>>> In the meantime is there any viable workaround? It seems like HA
>>>>>>>>>> doesn't really work due to this bug.
>>>>>>>>>>
>>>>>>>>>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> About some implementation mechanisms.
>>>>>>>>>>> Flink uses Zookeeper to store JobGraph (Job's description
>>>>>>>>>>> information and metadata) as a basis for Job recovery.
>>>>>>>>>>> However, previous implementations may cause this information to
>>>>>>>>>>> not be properly cleaned up because it is asynchronously deleted by a
>>>>>>>>>>> background thread.
>>>>>>>>>>>
>>>>>>>>>>> Thanks, vino.
>>>>>>>>>>>
>>>>>>>>>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>>>>>>>>>
>>>>>>>>>>>> Hi Encho,
>>>>>>>>>>>>
>>>>>>>>>>>> This is a problem already known to the Flink community, you can
>>>>>>>>>>>> track its progress through FLINK-10011[1], and currently Till is fixing
>>>>>>>>>>>> this issue.
>>>>>>>>>>>>
>>>>>>>>>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks, vino.
>>>>>>>>>>>>
>>>>>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一
>>>>>>>>>>>> 下午10:13写道:
>>>>>>>>>>>>
>>>>>>>>>>>>> I am running Flink 1.5.3 with two job managers and two task
>>>>>>>>>>>>> managers in Kubernetes along with HDFS and Zookeeper in high-availability
>>>>>>>>>>>>> mode.
>>>>>>>>>>>>>
>>>>>>>>>>>>> My problem occurs after the following actions:
>>>>>>>>>>>>> - Upload a .jar file to jobmanager-1
>>>>>>>>>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>>>>>>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>>>>>>>>>> - Kill pod of jobmanager-1
>>>>>>>>>>>>> After a short delay, jobmanager-2 takes leadership and
>>>>>>>>>>>>> correctly restores the job and continues it
>>>>>>>>>>>>> - Stop job from jobmanager-2
>>>>>>>>>>>>>
>>>>>>>>>>>>> At this point all seems well, but the problem is that
>>>>>>>>>>>>> jobmanager-2 does not clean up anything that was left from jobmanager-1.
>>>>>>>>>>>>> This means that both in HDFS and in Zookeeper remain job graphs, which
>>>>>>>>>>>>> later on obstruct any work of both managers as after any reset they
>>>>>>>>>>>>> unsuccessfully try to restore a non-existent job and fail over and over
>>>>>>>>>>>>> again.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I am quite certain that jobmanager-2 does not know about any
>>>>>>>>>>>>> of jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>>>>>>>>>> duplicate job folders:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>>>>>>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>>
>>>>>>>>>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in
>>>>>>>>>>>>> Zookeeper when the job is stopped, but fails since there are leftover files
>>>>>>>>>>>>> in it from jobmanager-1:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>>>>>>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>>
>>>>>>>>>>>>> I’ve noticed that when restoring the job, it seems like
>>>>>>>>>>>>> jobmanager-2 does not get anything more than jobID, while it perhaps needs
>>>>>>>>>>>>> some metadata? Here is the log that seems suspicious to me:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2018-08-27 13:09:18,113 INFO
>>>>>>>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>>>>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>>>>>>>>>
>>>>>>>>>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to
>>>>>>>>>>>>> be aware that it’s overwriting anything or not deleting properly.
>>>>>>>>>>>>>
>>>>>>>>>>>>> My question is - what is the intended way for the job managers
>>>>>>>>>>>>> to correctly exchange metadata in HA mode and why is it not working for me?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks in advance!
>>>>>>>>>>>>
>>>>>>>>>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by Till Rohrmann <tr...@apache.org>.
Hi Encho,

thanks for sending the first part of the logs. What I would actually be
interested in are the complete logs because somewhere in the jobmanager-2
logs there must be a log statement saying that the respective dispatcher
gained leadership. I would like to see why this happens but for this to
debug the complete logs are necessary. It would be awesome if you could
send them to me. Thanks a lot!

Cheers,
Till

On Wed, Aug 29, 2018 at 2:00 PM Encho Mishinev <en...@gmail.com>
wrote:

> Hi Till,
>
> I will use the approach with a k8s deployment and HA mode with a single
> job manager. Nonetheless, here are the logs I just produced by repeating
> the aforementioned experiment, hope they help in debugging:
>
> *- Starting Jobmanager-1:*
>
> Starting Job Manager
> sed: cannot rename /opt/flink/conf/sedR98XPn: Device or resource busy
> config file:
> jobmanager.rpc.address: flink-jobmanager-1
> jobmanager.rpc.port: 6123
> jobmanager.heap.size: 8192
> taskmanager.heap.size: 8192
> taskmanager.numberOfTaskSlots: 4
> high-availability: zookeeper
> high-availability.storageDir:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
> high-availability.zookeeper.quorum: zk-cs:2181
> high-availability.zookeeper.path.root: /flink
> high-availability.jobmanager.port: 50010
> state.backend: filesystem
> state.checkpoints.dir:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
> state.savepoints.dir:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
> state.backend.incremental: false
> fs.default-scheme:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
> rest.port: 8081
> web.upload.dir: /opt/flink/upload
> query.server.port: 6125
> taskmanager.numberOfTaskSlots: 4
> classloader.parent-first-patterns.additional: org.apache.xerces.
> blob.storage.directory: /opt/flink/blob-server
> blob.server.port: 6124
> blob.server.port: 6124
> query.server.port: 6125
> Starting standalonesession as a console application on host
> flink-jobmanager-1-f76fd4df8-ftwt9.
> 2018-08-29 11:41:48,806 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
> --------------------------------------------------------------------------------
> 2018-08-29 11:41:48,807 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
> StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
> Date:16.08.2018 @ 06:39:50 GMT)
> 2018-08-29 11:41:48,807 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
> user: flink
> 2018-08-29 11:41:49,134 WARN  org.apache.hadoop.util.NativeCodeLoader
>                  - Unable to load native-hadoop library for your
> platform... using builtin-java classes where applicable
> 2018-08-29 11:41:49,210 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
> Hadoop/Kerberos user: flink
> 2018-08-29 11:41:49,210 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
> 2018-08-29 11:41:49,210 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
> heap size: 6702 MiBytes
> 2018-08-29 11:41:49,210 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
> /docker-java-home/jre
> 2018-08-29 11:41:49,213 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
> version: 2.7.5
> 2018-08-29 11:41:49,213 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
> Options:
> 2018-08-29 11:41:49,213 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
> 2018-08-29 11:41:49,213 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
> 2018-08-29 11:41:49,213 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
> Arguments:
> 2018-08-29 11:41:49,213 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  --configDir
> 2018-08-29 11:41:49,213 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  /opt/flink/conf
> 2018-08-29 11:41:49,213 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  --executionMode
> 2018-08-29 11:41:49,213 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
> 2018-08-29 11:41:49,214 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
> 2018-08-29 11:41:49,214 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
> 2018-08-29 11:41:49,214 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
> /opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
> 2018-08-29 11:41:49,214 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
> --------------------------------------------------------------------------------
> 2018-08-29 11:41:49,215 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
> UNIX signal handlers for [TERM, HUP, INT]
> 2018-08-29 11:41:49,221 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: jobmanager.rpc.address, flink-jobmanager-1
> 2018-08-29 11:41:49,221 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: jobmanager.rpc.port, 6123
> 2018-08-29 11:41:49,221 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: jobmanager.heap.size, 8192
> 2018-08-29 11:41:49,221 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: taskmanager.heap.size, 8192
> 2018-08-29 11:41:49,221 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: taskmanager.numberOfTaskSlots, 4
> 2018-08-29 11:41:49,222 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability, zookeeper
> 2018-08-29 11:41:49,222 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.storageDir,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
> 2018-08-29 11:41:49,222 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.zookeeper.quorum, zk-cs:2181
> 2018-08-29 11:41:49,222 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.zookeeper.path.root, /flink
> 2018-08-29 11:41:49,223 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.jobmanager.port, 50010
> 2018-08-29 11:41:49,223 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.backend, filesystem
> 2018-08-29 11:41:49,223 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.checkpoints.dir,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
> 2018-08-29 11:41:49,223 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.savepoints.dir,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
> 2018-08-29 11:41:49,223 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.backend.incremental, false
> 2018-08-29 11:41:49,224 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: fs.default-scheme,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
> 2018-08-29 11:41:49,224 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: rest.port, 8081
> 2018-08-29 11:41:49,224 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: web.upload.dir, /opt/flink/upload
> 2018-08-29 11:41:49,224 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: query.server.port, 6125
> 2018-08-29 11:41:49,225 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: taskmanager.numberOfTaskSlots, 4
> 2018-08-29 11:41:49,225 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: classloader.parent-first-patterns.additional,
> org.apache.xerces.
> 2018-08-29 11:41:49,225 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: blob.storage.directory, /opt/flink/blob-server
> 2018-08-29 11:41:49,225 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: blob.server.port, 6124
> 2018-08-29 11:41:49,225 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: blob.server.port, 6124
> 2018-08-29 11:41:49,225 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: query.server.port, 6125
> 2018-08-29 11:41:49,239 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
> StandaloneSessionClusterEntrypoint.
> 2018-08-29 11:41:49,239 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
> default filesystem.
> 2018-08-29 11:41:49,250 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
> security context.
> 2018-08-29 11:41:49,282 INFO
> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
> set to flink (auth:SIMPLE)
> 2018-08-29 11:41:49,298 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
> Initializing cluster services.
> 2018-08-29 11:41:49,309 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
> start actor system at flink-jobmanager-1:50010
> 2018-08-29 11:41:49,768 INFO  akka.event.slf4j.Slf4jLogger
>                   - Slf4jLogger started
> 2018-08-29 11:41:49,823 INFO  akka.remote.Remoting
>                   - Starting remoting
> 2018-08-29 11:41:49,974 INFO  akka.remote.Remoting
>                   - Remoting started; listening on addresses
> :[akka.tcp://flink@flink-jobmanager-1:50010]
> 2018-08-29 11:41:49,981 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
> system started at akka.tcp://flink@flink-jobmanager-1:50010
> 2018-08-29 11:41:50,444 INFO
> org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
> highly available BLOB storage directory at
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
> 2018-08-29 11:41:50,509 INFO
> org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing
> default ACL for ZK connections
> 2018-08-29 11:41:50,509 INFO
> org.apache.flink.runtime.util.ZooKeeperUtils                  - Using
> '/flink/default' as Zookeeper namespace.
> 2018-08-29 11:41:50,568 INFO
> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
> - Starting
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
> built on 03/23/2017 10:13 GMT
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:host.name=flink-jobmanager-1-f76fd4df8-ftwt9
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.version=1.8.0_181
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.vendor=Oracle Corporation
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.io.tmpdir=/tmp
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.compiler=<NA>
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:os.name=Linux
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:os.arch=amd64
> 2018-08-29 11:41:50,577 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:os.version=4.4.0-1027-gke
> 2018-08-29 11:41:50,578 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:user.name=flink
> 2018-08-29 11:41:50,578 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:user.home=/opt/flink
> 2018-08-29 11:41:50,578 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:user.dir=/opt/flink
> 2018-08-29 11:41:50,578 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
> Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
> 2018-08-29 11:41:50,605 INFO  org.apache.flink.runtime.blob.BlobServer
>                   - Created BLOB server storage directory
> /opt/flink/blob-server/blobStore-d408cea8-2ed0-461a-a30a-a62b70fd332a
> 2018-08-29 11:41:50,605 WARN
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
> configuration failed: javax.security.auth.login.LoginException: No JAAS
> configuration section named 'Client' was found in specified JAAS
> configuration file: '/tmp/jaas-5372401662150571998.conf'. Will continue
> connection to Zookeeper server without SASL authentication, if Zookeeper
> server allows it.
> 2018-08-29 11:41:50,607 INFO  org.apache.flink.runtime.blob.BlobServer
>                   - Started BLOB server at 0.0.0.0:6124 - max concurrent
> requests: 50 - max backlog: 1000
> 2018-08-29 11:41:50,607 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
> Opening socket connection to server zk-cs.default.svc.cluster.local/
> 10.27.248.104:2181
> 2018-08-29 11:41:50,608 ERROR
> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
> Authentication failed
> 2018-08-29 11:41:50,609 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
> connection established to zk-cs.default.svc.cluster.local/
> 10.27.248.104:2181, initiating session
> 2018-08-29 11:41:50,618 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
> Session establishment complete on server zk-cs.default.svc.cluster.local/
> 10.27.248.104:2181, sessionid = 0x26584fd55690005, negotiated timeout =
> 40000
> 2018-08-29 11:41:50,619 INFO
> org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
> - State change: CONNECTED
> 2018-08-29 11:41:50,627 INFO
> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
> reporter configured, no metrics will be exposed/reported.
> 2018-08-29 11:41:50,633 INFO
> org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
> Initializing FileArchivedExecutionGraphStore: Storage directory
> /tmp/executionGraphStore-c5df0b39-86f3-4fba-bdda-aacca4f86086, expiration
> time 3600000, maximum cache size 52428800 bytes.
> 2018-08-29 11:41:50,659 INFO
> org.apache.flink.runtime.blob.TransientBlobCache              - Created
> BLOB cache storage directory
> /opt/flink/blob-server/blobStore-c12d55af-3c2d-4fc2-8ee8-6de642522184
> 2018-08-29 11:41:50,674 WARN
> org.apache.flink.configuration.Configuration                  - Config uses
> deprecated configuration key 'jobmanager.rpc.address' instead of proper key
> 'rest.address'
> 2018-08-29 11:41:50,675 WARN
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
> directory /opt/flink/upload/flink-web-upload does not exist, or has been
> deleted externally. Previously uploaded files are no longer available.
> 2018-08-29 11:41:50,676 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
> directory /opt/flink/upload/flink-web-upload for file uploads.
> 2018-08-29 11:41:50,679 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
> rest endpoint.
> 2018-08-29 11:41:50,995 WARN
> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
> environment variable 'log.file' is not set.
> 2018-08-29 11:41:50,995 WARN
> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
> log files are unavailable in the web dashboard. Log file location not found
> in environment variable 'log.file' or configuration key 'Key:
> 'web.log.path' , default: null (deprecated keys:
> [jobmanager.web.log.path])'.
> 2018-08-29 11:41:51,071 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
> endpoint listening at flink-jobmanager-1:8081
> 2018-08-29 11:41:51,071 INFO
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
> Starting ZooKeeperLeaderElectionService
> ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
> 2018-08-29 11:41:51,091 WARN
> org.apache.flink.shaded.curator.org.apache.curator.utils.ZKPaths  - The
> version of ZooKeeper being used doesn't support Container nodes.
> CreateMode.PERSISTENT will be used instead.
> 2018-08-29 11:41:51,101 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
> frontend listening at http://flink-jobmanager-1:8081.
> 2018-08-29 11:41:51,114 INFO
> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
> RPC endpoint for
> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
> akka://flink/user/resourcemanager .
> 2018-08-29 11:41:51,141 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    -
> http://flink-jobmanager-1:8081 was granted leadership with
> leaderSessionID=bb0d4dfd-c2c4-480b-bc86-62e231a606dd
> 2018-08-29 11:41:51,214 INFO
> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
> RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
> at akka://flink/user/dispatcher .
> 2018-08-29 11:41:51,230 INFO
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
> Starting ZooKeeperLeaderElectionService
> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
> 2018-08-29 11:41:51,232 INFO
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
> Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
> 2018-08-29 11:41:51,234 INFO
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
> Starting ZooKeeperLeaderElectionService
> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
> 2018-08-29 11:41:51,235 INFO
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
> 2018-08-29 11:41:51,253 INFO
> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  -
> ResourceManager akka.tcp://flink@flink-jobmanager-1:50010/user/resourcemanager
> was granted leadership with fencing token ba47ed8daa8ff16bea6fc355c13f4d49
> 2018-08-29 11:41:51,254 INFO
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
> Starting the SlotManager.
> 2018-08-29 11:41:51,263 INFO
> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Dispatcher
> akka.tcp://flink@flink-jobmanager-1:50010/user/dispatcher was granted
> leadership with fencing token 703301bf-85e7-4464-990f-ad39128a7b4d
> 2018-08-29 11:41:51,263 INFO
> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Recovering
> all persisted jobs.
> 2018-08-29 11:41:51,468 INFO
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
> Registering TaskManager c8a3201d58d87dbbe16f8eb352b5c5b6 under
> 1c5bf0bc3848bd384b6f032ff7213754 at the SlotManager.
> 2018-08-29 11:41:51,471 INFO
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
> Registering TaskManager 104d18b72fed054620e58e120a1ea083 under
> e9d3e8ad3b477dd2e58bcb88a2c0d061 at the SlotManager.
>
> *Starting Jobmanager-2:*
>
> Starting Job Manager
> sed: cannot rename /opt/flink/conf/sedH2ZiSu: Device or resource busy
> config file:
> jobmanager.rpc.address: flink-jobmanager-2
> jobmanager.rpc.port: 6123
> jobmanager.heap.size: 8192
> taskmanager.heap.size: 8192
> taskmanager.numberOfTaskSlots: 4
> high-availability: zookeeper
> high-availability.storageDir:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
> high-availability.zookeeper.quorum: zk-cs:2181
> high-availability.zookeeper.path.root: /flink
> high-availability.jobmanager.port: 50010
> state.backend: filesystem
> state.checkpoints.dir:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
> state.savepoints.dir:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
> state.backend.incremental: false
> fs.default-scheme:
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
> rest.port: 8081
> web.upload.dir: /opt/flink/upload
> query.server.port: 6125
> taskmanager.numberOfTaskSlots: 4
> classloader.parent-first-patterns.additional: org.apache.xerces.
> blob.storage.directory: /opt/flink/blob-server
> blob.server.port: 6124
> blob.server.port: 6124
> query.server.port: 6125
> Starting standalonesession as a console application on host
> flink-jobmanager-2-7844b78c9-kmvw9.
> 2018-08-29 11:41:51,688 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
> --------------------------------------------------------------------------------
> 2018-08-29 11:41:51,690 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
> StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
> Date:16.08.2018 @ 06:39:50 GMT)
> 2018-08-29 11:41:51,690 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
> user: flink
> 2018-08-29 11:41:52,018 WARN  org.apache.hadoop.util.NativeCodeLoader
>                  - Unable to load native-hadoop library for your
> platform... using builtin-java classes where applicable
> 2018-08-29 11:41:52,088 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
> Hadoop/Kerberos user: flink
> 2018-08-29 11:41:52,088 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
> 2018-08-29 11:41:52,088 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
> heap size: 6702 MiBytes
> 2018-08-29 11:41:52,088 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
> /docker-java-home/jre
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
> version: 2.7.5
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
> Options:
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
> Arguments:
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  --configDir
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  /opt/flink/conf
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>  --executionMode
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
> /opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
> 2018-08-29 11:41:52,091 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
> --------------------------------------------------------------------------------
> 2018-08-29 11:41:52,092 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
> UNIX signal handlers for [TERM, HUP, INT]
> 2018-08-29 11:41:52,103 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: jobmanager.rpc.address, flink-jobmanager-2
> 2018-08-29 11:41:52,103 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: jobmanager.rpc.port, 6123
> 2018-08-29 11:41:52,103 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: jobmanager.heap.size, 8192
> 2018-08-29 11:41:52,104 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: taskmanager.heap.size, 8192
> 2018-08-29 11:41:52,104 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: taskmanager.numberOfTaskSlots, 4
> 2018-08-29 11:41:52,104 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability, zookeeper
> 2018-08-29 11:41:52,104 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.storageDir,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
> 2018-08-29 11:41:52,104 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.zookeeper.quorum, zk-cs:2181
> 2018-08-29 11:41:52,104 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.zookeeper.path.root, /flink
> 2018-08-29 11:41:52,105 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: high-availability.jobmanager.port, 50010
> 2018-08-29 11:41:52,105 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.backend, filesystem
> 2018-08-29 11:41:52,105 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.checkpoints.dir,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
> 2018-08-29 11:41:52,105 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.savepoints.dir,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
> 2018-08-29 11:41:52,105 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: state.backend.incremental, false
> 2018-08-29 11:41:52,106 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: fs.default-scheme,
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
> 2018-08-29 11:41:52,106 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: rest.port, 8081
> 2018-08-29 11:41:52,106 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: web.upload.dir, /opt/flink/upload
> 2018-08-29 11:41:52,106 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: query.server.port, 6125
> 2018-08-29 11:41:52,106 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: taskmanager.numberOfTaskSlots, 4
> 2018-08-29 11:41:52,107 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: classloader.parent-first-patterns.additional,
> org.apache.xerces.
> 2018-08-29 11:41:52,107 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: blob.storage.directory, /opt/flink/blob-server
> 2018-08-29 11:41:52,107 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: blob.server.port, 6124
> 2018-08-29 11:41:52,107 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: blob.server.port, 6124
> 2018-08-29 11:41:52,107 INFO
> org.apache.flink.configuration.GlobalConfiguration            - Loading
> configuration property: query.server.port, 6125
> 2018-08-29 11:41:52,122 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
> StandaloneSessionClusterEntrypoint.
> 2018-08-29 11:41:52,123 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
> default filesystem.
> 2018-08-29 11:41:52,133 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
> security context.
> 2018-08-29 11:41:52,173 INFO
> org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
> set to flink (auth:SIMPLE)
> 2018-08-29 11:41:52,188 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
> Initializing cluster services.
> 2018-08-29 11:41:52,198 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
> start actor system at flink-jobmanager-2:50010
> 2018-08-29 11:41:52,753 INFO  akka.event.slf4j.Slf4jLogger
>                   - Slf4jLogger started
> 2018-08-29 11:41:52,822 INFO  akka.remote.Remoting
>                   - Starting remoting
> 2018-08-29 11:41:53,038 INFO  akka.remote.Remoting
>                   - Remoting started; listening on addresses
> :[akka.tcp://flink@flink-jobmanager-2:50010]
> 2018-08-29 11:41:53,046 INFO
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
> system started at akka.tcp://flink@flink-jobmanager-2:50010
> 2018-08-29 11:41:53,500 INFO
> org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
> highly available BLOB storage directory at
> hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
> 2018-08-29 11:41:53,558 INFO
> org.apache.flink.runtime.util.ZooKeeperUtils                  - Enforcing
> default ACL for ZK connections
> 2018-08-29 11:41:53,559 INFO
> org.apache.flink.runtime.util.ZooKeeperUtils                  - Using
> '/flink/default' as Zookeeper namespace.
> 2018-08-29 11:41:53,616 INFO
> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
> - Starting
> 2018-08-29 11:41:53,624 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
> built on 03/23/2017 10:13 GMT
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:host.name=flink-jobmanager-2-7844b78c9-kmvw9
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.version=1.8.0_181
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.vendor=Oracle Corporation
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.io.tmpdir=/tmp
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:java.compiler=<NA>
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:os.name=Linux
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:os.arch=amd64
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:os.version=4.4.0-1027-gke
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:user.name=flink
> 2018-08-29 11:41:53,625 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:user.home=/opt/flink
> 2018-08-29 11:41:53,626 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
> environment:user.dir=/opt/flink
> 2018-08-29 11:41:53,626 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
> Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
> 2018-08-29 11:41:53,644 WARN
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
> configuration failed: javax.security.auth.login.LoginException: No JAAS
> configuration section named 'Client' was found in specified JAAS
> configuration file: '/tmp/jaas-8238466329925822361.conf'. Will continue
> connection to Zookeeper server without SASL authentication, if Zookeeper
> server allows it.
> 2018-08-29 11:41:53,646 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
> Opening socket connection to server zk-cs.default.svc.cluster.local/
> 10.27.248.104:2181
> 2018-08-29 11:41:53,646 INFO  org.apache.flink.runtime.blob.BlobServer
>                   - Created BLOB server storage directory
> /opt/flink/blob-server/blobStore-61cdb645-5d0c-47fd-bcf6-84ad16fadade
> 2018-08-29 11:41:53,646 ERROR
> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
> Authentication failed
> 2018-08-29 11:41:53,647 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
> connection established to zk-cs.default.svc.cluster.local/
> 10.27.248.104:2181, initiating session
> 2018-08-29 11:41:53,649 INFO  org.apache.flink.runtime.blob.BlobServer
>                   - Started BLOB server at 0.0.0.0:6124 - max concurrent
> requests: 50 - max backlog: 1000
> 2018-08-29 11:41:53,655 INFO
> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
> Session establishment complete on server zk-cs.default.svc.cluster.local/
> 10.27.248.104:2181, sessionid = 0x26584fd55690006, negotiated timeout =
> 40000
> 2018-08-29 11:41:53,656 INFO
> org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
> - State change: CONNECTED
> 2018-08-29 11:41:53,667 INFO
> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
> reporter configured, no metrics will be exposed/reported.
> 2018-08-29 11:41:53,673 INFO
> org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
> Initializing FileArchivedExecutionGraphStore: Storage directory
> /tmp/executionGraphStore-8b236c14-79ee-4a84-b23f-437408c4661a, expiration
> time 3600000, maximum cache size 52428800 bytes.
> 2018-08-29 11:41:53,699 INFO
> org.apache.flink.runtime.blob.TransientBlobCache              - Created
> BLOB cache storage directory
> /opt/flink/blob-server/blobStore-80c519df-cc6f-4e9c-9cd5-da4077c826f0
> 2018-08-29 11:41:53,717 WARN
> org.apache.flink.configuration.Configuration                  - Config uses
> deprecated configuration key 'jobmanager.rpc.address' instead of proper key
> 'rest.address'
> 2018-08-29 11:41:53,718 WARN
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
> directory /opt/flink/upload/flink-web-upload does not exist, or has been
> deleted externally. Previously uploaded files are no longer available.
> 2018-08-29 11:41:53,719 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
> directory /opt/flink/upload/flink-web-upload for file uploads.
> 2018-08-29 11:41:53,722 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
> rest endpoint.
> 2018-08-29 11:41:54,084 WARN
> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
> environment variable 'log.file' is not set.
> 2018-08-29 11:41:54,084 WARN
> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
> log files are unavailable in the web dashboard. Log file location not found
> in environment variable 'log.file' or configuration key 'Key:
> 'web.log.path' , default: null (deprecated keys:
> [jobmanager.web.log.path])'.
> 2018-08-29 11:41:54,160 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
> endpoint listening at flink-jobmanager-2:8081
> 2018-08-29 11:41:54,160 INFO
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
> Starting ZooKeeperLeaderElectionService
> ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
> 2018-08-29 11:41:54,180 INFO
> org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
> frontend listening at http://flink-jobmanager-2:8081.
> 2018-08-29 11:41:54,192 INFO
> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
> RPC endpoint for
> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
> akka://flink/user/resourcemanager .
> 2018-08-29 11:41:54,273 INFO
> org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
> RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
> at akka://flink/user/dispatcher .
> 2018-08-29 11:41:54,286 INFO
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
> Starting ZooKeeperLeaderElectionService
> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
> 2018-08-29 11:41:54,287 INFO
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
> Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
> 2018-08-29 11:41:54,289 INFO
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
> Starting ZooKeeperLeaderElectionService
> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
> 2018-08-29 11:41:54,289 INFO
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
> Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>
> *Upon submitting a batch job on Jobmanager-1, we immediately get this log
> on Jobmanager-2*
> 2018-08-29 11:47:06,249 INFO
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
> Recovered SubmittedJobGraph(d69b67e4d28a2d244b06d3f6d661bca1, null).
>
> *Meanwhile Jobmanager-1 gets:*
> *-FlinkBatchPipelineTranslator pipeline logs- (we use Apache Beam)*
>
> 2018-08-29 11:47:06,006 INFO
> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Submitting
> job d69b67e4d28a2d244b06d3f6d661bca1
> (sicassandrawriterbeam-flink-0829114703-7d95fabd).
> 2018-08-29 11:47:06,090 INFO
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
> Added SubmittedJobGraph(d69b67e4d28a2d244b06d3f6d661bca1, null) to
> ZooKeeper.
>
> *-loads of job execution info-*
>
> 2018-08-29 11:49:20,272 INFO
> org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Job
> d69b67e4d28a2d244b06d3f6d661bca1 reached globally terminal state FINISHED.
> 2018-08-29 11:49:20,286 INFO
> org.apache.flink.runtime.jobmaster.JobMaster                  - Stopping
> the JobMaster for job
> sicassandrawriterbeam-flink-0829114703-7d95fabd(d69b67e4d28a2d244b06d3f6d661bca1).
> 2018-08-29 11:49:20,290 INFO
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
> Stopping ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
> 2018-08-29 11:49:20,292 INFO
> org.apache.flink.runtime.jobmaster.JobMaster                  - Close
> ResourceManager connection 827b94881bf7c94d8516907e04e3a564: JobManager is
> shutting down..
> 2018-08-29 11:49:20,292 INFO
> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Suspending
> SlotPool.
> 2018-08-29 11:49:20,293 INFO
> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Stopping
> SlotPool.
> 2018-08-29 11:49:20,293 INFO
> org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  -
> Disconnect job manager a3dab0a0883c5f0f37943358d9104d79
> @akka.tcp://flink@flink-jobmanager-1:50010/user/jobmanager_0 for job
> d69b67e4d28a2d244b06d3f6d661bca1 from the resource manager.
> 2018-08-29 11:49:20,293 INFO
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
> Stopping ZooKeeperLeaderElectionService
> ZooKeeperLeaderElectionService{leaderPath='/leader/d69b67e4d28a2d244b06d3f6d661bca1/job_manager_lock'}.
> 2018-08-29 11:49:20,304 INFO
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
> Removed job graph d69b67e4d28a2d244b06d3f6d661bca1 from ZooKeeper.
>
>
> -------------------
>
> The result is:
> HDFS has only a jobgraph and an empty default folder - everything else is
> cleared
> ZooKeeper has the jobgraph that Jobmanager-1 claims to have removed in the
> last log still there.
>
> On Wed, Aug 29, 2018 at 12:14 PM Till Rohrmann <tr...@apache.org>
> wrote:
>
>> Hi Encho,
>>
>> it sounds strange that the standby JobManager tries to recover a
>> submitted job graph. This should only happen if it has been granted
>> leadership. Thus, it seems as if the standby JobManager thinks that it is
>> also the leader. Could you maybe share the logs of the two
>> JobManagers/ClusterEntrypoints with us?
>>
>> Running only a single JobManager/ClusterEntrypoint in HA mode via a
>> Kubernetes Deployment should do the trick and there is nothing wrong with
>> it.
>>
>> Cheers,
>> Till
>>
>> On Wed, Aug 29, 2018 at 11:05 AM Encho Mishinev <en...@gmail.com>
>> wrote:
>>
>>> Hello,
>>>
>>> Since two job managers don't seem to be working for me I was thinking of
>>> just using a single job manager in Kubernetes in HA mode with a deployment
>>> ensuring its restart whenever it fails. Is this approach viable? The
>>> High-Availability page mentions that you use only one job manager in an
>>> YARN cluster but does not specify such option for Kubernetes. Is there
>>> anything that can go wrong with this approach?
>>>
>>> Thanks
>>>
>>> On Wed, Aug 29, 2018 at 11:10 AM Encho Mishinev <
>>> encho.mishinev@gmail.com> wrote:
>>>
>>>> Hi,
>>>>
>>>> Unfortunately the thing I described does indeed happen every time. As
>>>> mentioned in the first email, I am running on Kubernetes so certain things
>>>> could be different compared to just a standalone cluster.
>>>>
>>>> Any ideas for workarounds are welcome, as this problem basically
>>>> prevents me from using HA.
>>>>
>>>> Thanks,
>>>> Encho
>>>>
>>>> On Wed, Aug 29, 2018 at 5:15 AM vino yang <ya...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi Encho,
>>>>>
>>>>> From your description, I feel that there are extra bugs.
>>>>>
>>>>> About your description:
>>>>>
>>>>> *- Start both job managers*
>>>>> *- Start a batch job in JobManager 1 and let it finish*
>>>>> *The jobgraphs in both Zookeeper and HDFS remained.*
>>>>>
>>>>> Is it necessarily happening every time?
>>>>>
>>>>> In the Standalone cluster, the problems we encountered were sporadic.
>>>>>
>>>>> Thanks, vino.
>>>>>
>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午8:07写道:
>>>>>
>>>>>> Hello Till,
>>>>>>
>>>>>> I spend a few more hours testing and looking at the logs and it seems
>>>>>> like there's a more general problem here. While the two job managers are
>>>>>> active neither of them can properly delete jobgraphs. The above problem I
>>>>>> described comes from the fact that Kubernetes gets JobManager 1 quickly
>>>>>> after I manually kill it, so when I stop the job on JobManager 2 both are
>>>>>> alive.
>>>>>>
>>>>>> I did a very simple test:
>>>>>>
>>>>>> - Start both job managers
>>>>>> - Start a batch job in JobManager 1 and let it finish
>>>>>> The jobgraphs in both Zookeeper and HDFS remained.
>>>>>>
>>>>>> On the other hand if we do:
>>>>>>
>>>>>> - Start only JobManager 1 (again in HA mode)
>>>>>> - Start a batch job and let it finish
>>>>>> The jobgraphs in both Zookeeper and HDFS are deleted fine.
>>>>>>
>>>>>> It seems like the standby manager still leaves some kind of lock on
>>>>>> the jobgraphs. Do you think that's possible? Have you seen a similar
>>>>>> problem?
>>>>>> The only logs that appear on the standby manager while waiting are of
>>>>>> the type:
>>>>>>
>>>>>> 2018-08-28 11:54:10,789 INFO
>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>> Recovered SubmittedJobGraph(9e0a109b57511930c95d3b54574a66e3, null).
>>>>>>
>>>>>> Note that this log appears on the standby jobmanager immediately when
>>>>>> a new job is submitted to the active jobmanager.
>>>>>> Also note that the blobs and checkpoints are cleared fine. The
>>>>>> problem is only for jobgraphs both in ZooKeeper and HDFS.
>>>>>>
>>>>>> Trying to access the UI of the standby manager redirects to the
>>>>>> active one, so it is not a problem of them not knowing who the leader is.
>>>>>> Do you have any ideas?
>>>>>>
>>>>>> Thanks a lot,
>>>>>> Encho
>>>>>>
>>>>>> On Tue, Aug 28, 2018 at 10:27 AM Till Rohrmann <tr...@apache.org>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi Encho,
>>>>>>>
>>>>>>> thanks a lot for reporting this issue. The problem arises whenever
>>>>>>> the old leader maintains the connection to ZooKeeper. If this is the case,
>>>>>>> then ephemeral nodes which we create to protect against faulty delete
>>>>>>> operations are not removed and consequently the new leader is not able to
>>>>>>> delete the persisted job graph. So one thing to check is whether the old JM
>>>>>>> still has an open connection to ZooKeeper. The next thing to check is the
>>>>>>> session timeout of your ZooKeeper cluster. If you stop the job within the
>>>>>>> session timeout, then it is also not guaranteed that ZooKeeper has detected
>>>>>>> that the ephemeral nodes of the old JM must be deleted. In order to
>>>>>>> understand this better it would be helpful if you could tell us the timing
>>>>>>> of the different actions.
>>>>>>>
>>>>>>> Cheers,
>>>>>>> Till
>>>>>>>
>>>>>>> On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Encho,
>>>>>>>>
>>>>>>>> A temporary solution can be used to determine if it has been
>>>>>>>> cleaned up by monitoring the specific JobID under Zookeeper's "/jobgraph".
>>>>>>>> Another solution, modify the source code, rudely modify the cleanup
>>>>>>>> mode to the synchronous form, but the flink operation Zookeeper's path
>>>>>>>> needs to obtain the corresponding lock, so it is dangerous to do so, and it
>>>>>>>> is not recommended.
>>>>>>>> I think maybe this problem can be solved in the next version. It
>>>>>>>> depends on Till.
>>>>>>>>
>>>>>>>> Thanks, vino.
>>>>>>>>
>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:
>>>>>>>>
>>>>>>>>> Thank you very much for the info! Will keep track of the progress.
>>>>>>>>>
>>>>>>>>> In the meantime is there any viable workaround? It seems like HA
>>>>>>>>> doesn't really work due to this bug.
>>>>>>>>>
>>>>>>>>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> About some implementation mechanisms.
>>>>>>>>>> Flink uses Zookeeper to store JobGraph (Job's description
>>>>>>>>>> information and metadata) as a basis for Job recovery.
>>>>>>>>>> However, previous implementations may cause this information to
>>>>>>>>>> not be properly cleaned up because it is asynchronously deleted by a
>>>>>>>>>> background thread.
>>>>>>>>>>
>>>>>>>>>> Thanks, vino.
>>>>>>>>>>
>>>>>>>>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>>>>>>>>
>>>>>>>>>>> Hi Encho,
>>>>>>>>>>>
>>>>>>>>>>> This is a problem already known to the Flink community, you can
>>>>>>>>>>> track its progress through FLINK-10011[1], and currently Till is fixing
>>>>>>>>>>> this issue.
>>>>>>>>>>>
>>>>>>>>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>>>>>>>>
>>>>>>>>>>> Thanks, vino.
>>>>>>>>>>>
>>>>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一
>>>>>>>>>>> 下午10:13写道:
>>>>>>>>>>>
>>>>>>>>>>>> I am running Flink 1.5.3 with two job managers and two task
>>>>>>>>>>>> managers in Kubernetes along with HDFS and Zookeeper in high-availability
>>>>>>>>>>>> mode.
>>>>>>>>>>>>
>>>>>>>>>>>> My problem occurs after the following actions:
>>>>>>>>>>>> - Upload a .jar file to jobmanager-1
>>>>>>>>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>>>>>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>>>>>>>>> - Kill pod of jobmanager-1
>>>>>>>>>>>> After a short delay, jobmanager-2 takes leadership and
>>>>>>>>>>>> correctly restores the job and continues it
>>>>>>>>>>>> - Stop job from jobmanager-2
>>>>>>>>>>>>
>>>>>>>>>>>> At this point all seems well, but the problem is that
>>>>>>>>>>>> jobmanager-2 does not clean up anything that was left from jobmanager-1.
>>>>>>>>>>>> This means that both in HDFS and in Zookeeper remain job graphs, which
>>>>>>>>>>>> later on obstruct any work of both managers as after any reset they
>>>>>>>>>>>> unsuccessfully try to restore a non-existent job and fail over and over
>>>>>>>>>>>> again.
>>>>>>>>>>>>
>>>>>>>>>>>> I am quite certain that jobmanager-2 does not know about any of
>>>>>>>>>>>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>>>>>>>>> duplicate job folders:
>>>>>>>>>>>>
>>>>>>>>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>>>
>>>>>>>>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>>>>>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>
>>>>>>>>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in
>>>>>>>>>>>> Zookeeper when the job is stopped, but fails since there are leftover files
>>>>>>>>>>>> in it from jobmanager-1:
>>>>>>>>>>>>
>>>>>>>>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>>>>>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>>
>>>>>>>>>>>> I’ve noticed that when restoring the job, it seems like
>>>>>>>>>>>> jobmanager-2 does not get anything more than jobID, while it perhaps needs
>>>>>>>>>>>> some metadata? Here is the log that seems suspicious to me:
>>>>>>>>>>>>
>>>>>>>>>>>> 2018-08-27 13:09:18,113 INFO
>>>>>>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>>>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>>>>>>>>
>>>>>>>>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to be
>>>>>>>>>>>> aware that it’s overwriting anything or not deleting properly.
>>>>>>>>>>>>
>>>>>>>>>>>> My question is - what is the intended way for the job managers
>>>>>>>>>>>> to correctly exchange metadata in HA mode and why is it not working for me?
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks in advance!
>>>>>>>>>>>
>>>>>>>>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by Encho Mishinev <en...@gmail.com>.
Hi Till,

I will use the approach with a k8s deployment and HA mode with a single job
manager. Nonetheless, here are the logs I just produced by repeating the
aforementioned experiment, hope they help in debugging:

*- Starting Jobmanager-1:*

Starting Job Manager
sed: cannot rename /opt/flink/conf/sedR98XPn: Device or resource busy
config file:
jobmanager.rpc.address: flink-jobmanager-1
jobmanager.rpc.port: 6123
jobmanager.heap.size: 8192
taskmanager.heap.size: 8192
taskmanager.numberOfTaskSlots: 4
high-availability: zookeeper
high-availability.storageDir:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
high-availability.zookeeper.quorum: zk-cs:2181
high-availability.zookeeper.path.root: /flink
high-availability.jobmanager.port: 50010
state.backend: filesystem
state.checkpoints.dir:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
state.savepoints.dir:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
state.backend.incremental: false
fs.default-scheme:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
rest.port: 8081
web.upload.dir: /opt/flink/upload
query.server.port: 6125
taskmanager.numberOfTaskSlots: 4
classloader.parent-first-patterns.additional: org.apache.xerces.
blob.storage.directory: /opt/flink/blob-server
blob.server.port: 6124
blob.server.port: 6124
query.server.port: 6125
Starting standalonesession as a console application on host
flink-jobmanager-1-f76fd4df8-ftwt9.
2018-08-29 11:41:48,806 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
--------------------------------------------------------------------------------
2018-08-29 11:41:48,807 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
Date:16.08.2018 @ 06:39:50 GMT)
2018-08-29 11:41:48,807 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
user: flink
2018-08-29 11:41:49,134 WARN  org.apache.hadoop.util.NativeCodeLoader
                 - Unable to load native-hadoop library for your
platform... using builtin-java classes where applicable
2018-08-29 11:41:49,210 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
Hadoop/Kerberos user: flink
2018-08-29 11:41:49,210 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
2018-08-29 11:41:49,210 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
heap size: 6702 MiBytes
2018-08-29 11:41:49,210 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
/docker-java-home/jre
2018-08-29 11:41:49,213 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
version: 2.7.5
2018-08-29 11:41:49,213 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
Options:
2018-08-29 11:41:49,213 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
2018-08-29 11:41:49,213 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
2018-08-29 11:41:49,213 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
Arguments:
2018-08-29 11:41:49,213 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 --configDir
2018-08-29 11:41:49,213 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 /opt/flink/conf
2018-08-29 11:41:49,213 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 --executionMode
2018-08-29 11:41:49,213 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
2018-08-29 11:41:49,214 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
2018-08-29 11:41:49,214 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
2018-08-29 11:41:49,214 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
2018-08-29 11:41:49,214 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
--------------------------------------------------------------------------------
2018-08-29 11:41:49,215 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
UNIX signal handlers for [TERM, HUP, INT]
2018-08-29 11:41:49,221 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: jobmanager.rpc.address, flink-jobmanager-1
2018-08-29 11:41:49,221 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: jobmanager.rpc.port, 6123
2018-08-29 11:41:49,221 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: jobmanager.heap.size, 8192
2018-08-29 11:41:49,221 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: taskmanager.heap.size, 8192
2018-08-29 11:41:49,221 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: taskmanager.numberOfTaskSlots, 4
2018-08-29 11:41:49,222 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability, zookeeper
2018-08-29 11:41:49,222 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.storageDir,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
2018-08-29 11:41:49,222 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.zookeeper.quorum, zk-cs:2181
2018-08-29 11:41:49,222 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.zookeeper.path.root, /flink
2018-08-29 11:41:49,223 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.jobmanager.port, 50010
2018-08-29 11:41:49,223 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.backend, filesystem
2018-08-29 11:41:49,223 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.checkpoints.dir,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
2018-08-29 11:41:49,223 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.savepoints.dir,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
2018-08-29 11:41:49,223 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.backend.incremental, false
2018-08-29 11:41:49,224 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: fs.default-scheme,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
2018-08-29 11:41:49,224 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: rest.port, 8081
2018-08-29 11:41:49,224 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: web.upload.dir, /opt/flink/upload
2018-08-29 11:41:49,224 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: query.server.port, 6125
2018-08-29 11:41:49,225 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: taskmanager.numberOfTaskSlots, 4
2018-08-29 11:41:49,225 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: classloader.parent-first-patterns.additional,
org.apache.xerces.
2018-08-29 11:41:49,225 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: blob.storage.directory, /opt/flink/blob-server
2018-08-29 11:41:49,225 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: blob.server.port, 6124
2018-08-29 11:41:49,225 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: blob.server.port, 6124
2018-08-29 11:41:49,225 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: query.server.port, 6125
2018-08-29 11:41:49,239 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
StandaloneSessionClusterEntrypoint.
2018-08-29 11:41:49,239 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
default filesystem.
2018-08-29 11:41:49,250 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
security context.
2018-08-29 11:41:49,282 INFO
org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
set to flink (auth:SIMPLE)
2018-08-29 11:41:49,298 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
Initializing cluster services.
2018-08-29 11:41:49,309 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
start actor system at flink-jobmanager-1:50010
2018-08-29 11:41:49,768 INFO  akka.event.slf4j.Slf4jLogger
                - Slf4jLogger started
2018-08-29 11:41:49,823 INFO  akka.remote.Remoting
                - Starting remoting
2018-08-29 11:41:49,974 INFO  akka.remote.Remoting
                - Remoting started; listening on addresses
:[akka.tcp://flink@flink-jobmanager-1:50010]
2018-08-29 11:41:49,981 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
system started at akka.tcp://flink@flink-jobmanager-1:50010
2018-08-29 11:41:50,444 INFO
org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
highly available BLOB storage directory at
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
2018-08-29 11:41:50,509 INFO  org.apache.flink.runtime.util.ZooKeeperUtils
                - Enforcing default ACL for ZK connections
2018-08-29 11:41:50,509 INFO  org.apache.flink.runtime.util.ZooKeeperUtils
                - Using '/flink/default' as Zookeeper namespace.
2018-08-29 11:41:50,568 INFO
org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
- Starting
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
built on 03/23/2017 10:13 GMT
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:host.name=flink-jobmanager-1-f76fd4df8-ftwt9
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.version=1.8.0_181
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.vendor=Oracle Corporation
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.io.tmpdir=/tmp
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.compiler=<NA>
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:os.name=Linux
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:os.arch=amd64
2018-08-29 11:41:50,577 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:os.version=4.4.0-1027-gke
2018-08-29 11:41:50,578 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:user.name=flink
2018-08-29 11:41:50,578 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:user.home=/opt/flink
2018-08-29 11:41:50,578 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:user.dir=/opt/flink
2018-08-29 11:41:50,578 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
2018-08-29 11:41:50,605 INFO  org.apache.flink.runtime.blob.BlobServer
                - Created BLOB server storage directory
/opt/flink/blob-server/blobStore-d408cea8-2ed0-461a-a30a-a62b70fd332a
2018-08-29 11:41:50,605 WARN
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
configuration failed: javax.security.auth.login.LoginException: No JAAS
configuration section named 'Client' was found in specified JAAS
configuration file: '/tmp/jaas-5372401662150571998.conf'. Will continue
connection to Zookeeper server without SASL authentication, if Zookeeper
server allows it.
2018-08-29 11:41:50,607 INFO  org.apache.flink.runtime.blob.BlobServer
                - Started BLOB server at 0.0.0.0:6124 - max concurrent
requests: 50 - max backlog: 1000
2018-08-29 11:41:50,607 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
Opening socket connection to server zk-cs.default.svc.cluster.local/
10.27.248.104:2181
2018-08-29 11:41:50,608 ERROR
org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
Authentication failed
2018-08-29 11:41:50,609 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
connection established to zk-cs.default.svc.cluster.local/10.27.248.104:2181,
initiating session
2018-08-29 11:41:50,618 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
Session establishment complete on server zk-cs.default.svc.cluster.local/
10.27.248.104:2181, sessionid = 0x26584fd55690005, negotiated timeout =
40000
2018-08-29 11:41:50,619 INFO
org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
- State change: CONNECTED
2018-08-29 11:41:50,627 INFO
org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
reporter configured, no metrics will be exposed/reported.
2018-08-29 11:41:50,633 INFO
org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
Initializing FileArchivedExecutionGraphStore: Storage directory
/tmp/executionGraphStore-c5df0b39-86f3-4fba-bdda-aacca4f86086, expiration
time 3600000, maximum cache size 52428800 bytes.
2018-08-29 11:41:50,659 INFO
org.apache.flink.runtime.blob.TransientBlobCache              - Created
BLOB cache storage directory
/opt/flink/blob-server/blobStore-c12d55af-3c2d-4fc2-8ee8-6de642522184
2018-08-29 11:41:50,674 WARN  org.apache.flink.configuration.Configuration
                - Config uses deprecated configuration key
'jobmanager.rpc.address' instead of proper key 'rest.address'
2018-08-29 11:41:50,675 WARN
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
directory /opt/flink/upload/flink-web-upload does not exist, or has been
deleted externally. Previously uploaded files are no longer available.
2018-08-29 11:41:50,676 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
directory /opt/flink/upload/flink-web-upload for file uploads.
2018-08-29 11:41:50,679 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
rest endpoint.
2018-08-29 11:41:50,995 WARN
org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
environment variable 'log.file' is not set.
2018-08-29 11:41:50,995 WARN
org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
log files are unavailable in the web dashboard. Log file location not found
in environment variable 'log.file' or configuration key 'Key:
'web.log.path' , default: null (deprecated keys:
[jobmanager.web.log.path])'.
2018-08-29 11:41:51,071 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
endpoint listening at flink-jobmanager-1:8081
2018-08-29 11:41:51,071 INFO
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
Starting ZooKeeperLeaderElectionService
ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
2018-08-29 11:41:51,091 WARN
org.apache.flink.shaded.curator.org.apache.curator.utils.ZKPaths  - The
version of ZooKeeper being used doesn't support Container nodes.
CreateMode.PERSISTENT will be used instead.
2018-08-29 11:41:51,101 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
frontend listening at http://flink-jobmanager-1:8081.
2018-08-29 11:41:51,114 INFO
org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
RPC endpoint for
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
akka://flink/user/resourcemanager .
2018-08-29 11:41:51,141 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    -
http://flink-jobmanager-1:8081 was granted leadership with
leaderSessionID=bb0d4dfd-c2c4-480b-bc86-62e231a606dd
2018-08-29 11:41:51,214 INFO
org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
at akka://flink/user/dispatcher .
2018-08-29 11:41:51,230 INFO
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
Starting ZooKeeperLeaderElectionService
ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
2018-08-29 11:41:51,232 INFO
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2018-08-29 11:41:51,234 INFO
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
Starting ZooKeeperLeaderElectionService
ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
2018-08-29 11:41:51,235 INFO
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
2018-08-29 11:41:51,253 INFO
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  -
ResourceManager akka.tcp://flink@flink-jobmanager-1:50010/user/resourcemanager
was granted leadership with fencing token ba47ed8daa8ff16bea6fc355c13f4d49
2018-08-29 11:41:51,254 INFO
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
Starting the SlotManager.
2018-08-29 11:41:51,263 INFO
org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Dispatcher
akka.tcp://flink@flink-jobmanager-1:50010/user/dispatcher was granted
leadership with fencing token 703301bf-85e7-4464-990f-ad39128a7b4d
2018-08-29 11:41:51,263 INFO
org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Recovering
all persisted jobs.
2018-08-29 11:41:51,468 INFO
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
Registering TaskManager c8a3201d58d87dbbe16f8eb352b5c5b6 under
1c5bf0bc3848bd384b6f032ff7213754 at the SlotManager.
2018-08-29 11:41:51,471 INFO
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  -
Registering TaskManager 104d18b72fed054620e58e120a1ea083 under
e9d3e8ad3b477dd2e58bcb88a2c0d061 at the SlotManager.

*Starting Jobmanager-2:*

Starting Job Manager
sed: cannot rename /opt/flink/conf/sedH2ZiSu: Device or resource busy
config file:
jobmanager.rpc.address: flink-jobmanager-2
jobmanager.rpc.port: 6123
jobmanager.heap.size: 8192
taskmanager.heap.size: 8192
taskmanager.numberOfTaskSlots: 4
high-availability: zookeeper
high-availability.storageDir:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
high-availability.zookeeper.quorum: zk-cs:2181
high-availability.zookeeper.path.root: /flink
high-availability.jobmanager.port: 50010
state.backend: filesystem
state.checkpoints.dir:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
state.savepoints.dir:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
state.backend.incremental: false
fs.default-scheme:
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
rest.port: 8081
web.upload.dir: /opt/flink/upload
query.server.port: 6125
taskmanager.numberOfTaskSlots: 4
classloader.parent-first-patterns.additional: org.apache.xerces.
blob.storage.directory: /opt/flink/blob-server
blob.server.port: 6124
blob.server.port: 6124
query.server.port: 6125
Starting standalonesession as a console application on host
flink-jobmanager-2-7844b78c9-kmvw9.
2018-08-29 11:41:51,688 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
--------------------------------------------------------------------------------
2018-08-29 11:41:51,690 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Starting
StandaloneSessionClusterEntrypoint (Version: 1.5.3, Rev:614f216,
Date:16.08.2018 @ 06:39:50 GMT)
2018-08-29 11:41:51,690 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  OS current
user: flink
2018-08-29 11:41:52,018 WARN  org.apache.hadoop.util.NativeCodeLoader
                 - Unable to load native-hadoop library for your
platform... using builtin-java classes where applicable
2018-08-29 11:41:52,088 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Current
Hadoop/Kerberos user: flink
2018-08-29 11:41:52,088 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM:
OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.181-b13
2018-08-29 11:41:52,088 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Maximum
heap size: 6702 MiBytes
2018-08-29 11:41:52,088 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JAVA_HOME:
/docker-java-home/jre
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Hadoop
version: 2.7.5
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  JVM
Options:
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 -Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 -Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Program
Arguments:
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 --configDir
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 /opt/flink/conf
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
 --executionMode
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     --host
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -     cluster
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -  Classpath:
/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
2018-08-29 11:41:52,091 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
--------------------------------------------------------------------------------
2018-08-29 11:41:52,092 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Registered
UNIX signal handlers for [TERM, HUP, INT]
2018-08-29 11:41:52,103 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: jobmanager.rpc.address, flink-jobmanager-2
2018-08-29 11:41:52,103 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: jobmanager.rpc.port, 6123
2018-08-29 11:41:52,103 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: jobmanager.heap.size, 8192
2018-08-29 11:41:52,104 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: taskmanager.heap.size, 8192
2018-08-29 11:41:52,104 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: taskmanager.numberOfTaskSlots, 4
2018-08-29 11:41:52,104 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability, zookeeper
2018-08-29 11:41:52,104 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.storageDir,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability
2018-08-29 11:41:52,104 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.zookeeper.quorum, zk-cs:2181
2018-08-29 11:41:52,104 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.zookeeper.path.root, /flink
2018-08-29 11:41:52,105 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: high-availability.jobmanager.port, 50010
2018-08-29 11:41:52,105 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.backend, filesystem
2018-08-29 11:41:52,105 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.checkpoints.dir,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/checkpoints
2018-08-29 11:41:52,105 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.savepoints.dir,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/savepoints
2018-08-29 11:41:52,105 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: state.backend.incremental, false
2018-08-29 11:41:52,106 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: fs.default-scheme,
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020
2018-08-29 11:41:52,106 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: rest.port, 8081
2018-08-29 11:41:52,106 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: web.upload.dir, /opt/flink/upload
2018-08-29 11:41:52,106 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: query.server.port, 6125
2018-08-29 11:41:52,106 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: taskmanager.numberOfTaskSlots, 4
2018-08-29 11:41:52,107 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: classloader.parent-first-patterns.additional,
org.apache.xerces.
2018-08-29 11:41:52,107 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: blob.storage.directory, /opt/flink/blob-server
2018-08-29 11:41:52,107 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: blob.server.port, 6124
2018-08-29 11:41:52,107 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: blob.server.port, 6124
2018-08-29 11:41:52,107 INFO
org.apache.flink.configuration.GlobalConfiguration            - Loading
configuration property: query.server.port, 6125
2018-08-29 11:41:52,122 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Starting
StandaloneSessionClusterEntrypoint.
2018-08-29 11:41:52,123 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
default filesystem.
2018-08-29 11:41:52,133 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Install
security context.
2018-08-29 11:41:52,173 INFO
org.apache.flink.runtime.security.modules.HadoopModule        - Hadoop user
set to flink (auth:SIMPLE)
2018-08-29 11:41:52,188 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
Initializing cluster services.
2018-08-29 11:41:52,198 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Trying to
start actor system at flink-jobmanager-2:50010
2018-08-29 11:41:52,753 INFO  akka.event.slf4j.Slf4jLogger
                - Slf4jLogger started
2018-08-29 11:41:52,822 INFO  akka.remote.Remoting
                - Starting remoting
2018-08-29 11:41:53,038 INFO  akka.remote.Remoting
                - Remoting started; listening on addresses
:[akka.tcp://flink@flink-jobmanager-2:50010]
2018-08-29 11:41:53,046 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Actor
system started at akka.tcp://flink@flink-jobmanager-2:50010
2018-08-29 11:41:53,500 INFO
org.apache.flink.runtime.blob.FileSystemBlobStore             - Creating
highly available BLOB storage directory at
hdfs://hdfs-namenode-0.hdfs-namenode.default.svc.cluster.local:8020/flink/high-availability//default/blob
2018-08-29 11:41:53,558 INFO  org.apache.flink.runtime.util.ZooKeeperUtils
                - Enforcing default ACL for ZK connections
2018-08-29 11:41:53,559 INFO  org.apache.flink.runtime.util.ZooKeeperUtils
                - Using '/flink/default' as Zookeeper namespace.
2018-08-29 11:41:53,616 INFO
org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl
- Starting
2018-08-29 11:41:53,624 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
built on 03/23/2017 10:13 GMT
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:host.name=flink-jobmanager-2-7844b78c9-kmvw9
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.version=1.8.0_181
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.vendor=Oracle Corporation
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.home=/usr/lib/jvm/java-8-openjdk-amd64/jre
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.class.path=/opt/flink/lib/flink-python_2.11-1.5.3.jar:/opt/flink/lib/flink-shaded-hadoop2-uber-1.5.3.jar:/opt/flink/lib/log4j-1.2.17.jar:/opt/flink/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/lib/flink-dist_2.11-1.5.3.jar:::
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.io.tmpdir=/tmp
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:java.compiler=<NA>
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:os.name=Linux
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:os.arch=amd64
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:os.version=4.4.0-1027-gke
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:user.name=flink
2018-08-29 11:41:53,625 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:user.home=/opt/flink
2018-08-29 11:41:53,626 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
environment:user.dir=/opt/flink
2018-08-29 11:41:53,626 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  -
Initiating client connection, connectString=zk-cs:2181 sessionTimeout=60000
watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@17ae7628
2018-08-29 11:41:53,644 WARN
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL
configuration failed: javax.security.auth.login.LoginException: No JAAS
configuration section named 'Client' was found in specified JAAS
configuration file: '/tmp/jaas-8238466329925822361.conf'. Will continue
connection to Zookeeper server without SASL authentication, if Zookeeper
server allows it.
2018-08-29 11:41:53,646 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
Opening socket connection to server zk-cs.default.svc.cluster.local/
10.27.248.104:2181
2018-08-29 11:41:53,646 INFO  org.apache.flink.runtime.blob.BlobServer
                - Created BLOB server storage directory
/opt/flink/blob-server/blobStore-61cdb645-5d0c-47fd-bcf6-84ad16fadade
2018-08-29 11:41:53,646 ERROR
org.apache.flink.shaded.curator.org.apache.curator.ConnectionState  -
Authentication failed
2018-08-29 11:41:53,647 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
connection established to zk-cs.default.svc.cluster.local/10.27.248.104:2181,
initiating session
2018-08-29 11:41:53,649 INFO  org.apache.flink.runtime.blob.BlobServer
                - Started BLOB server at 0.0.0.0:6124 - max concurrent
requests: 50 - max backlog: 1000
2018-08-29 11:41:53,655 INFO
org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  -
Session establishment complete on server zk-cs.default.svc.cluster.local/
10.27.248.104:2181, sessionid = 0x26584fd55690006, negotiated timeout =
40000
2018-08-29 11:41:53,656 INFO
org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager
- State change: CONNECTED
2018-08-29 11:41:53,667 INFO
org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics
reporter configured, no metrics will be exposed/reported.
2018-08-29 11:41:53,673 INFO
org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore  -
Initializing FileArchivedExecutionGraphStore: Storage directory
/tmp/executionGraphStore-8b236c14-79ee-4a84-b23f-437408c4661a, expiration
time 3600000, maximum cache size 52428800 bytes.
2018-08-29 11:41:53,699 INFO
org.apache.flink.runtime.blob.TransientBlobCache              - Created
BLOB cache storage directory
/opt/flink/blob-server/blobStore-80c519df-cc6f-4e9c-9cd5-da4077c826f0
2018-08-29 11:41:53,717 WARN  org.apache.flink.configuration.Configuration
                - Config uses deprecated configuration key
'jobmanager.rpc.address' instead of proper key 'rest.address'
2018-08-29 11:41:53,718 WARN
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Upload
directory /opt/flink/upload/flink-web-upload does not exist, or has been
deleted externally. Previously uploaded files are no longer available.
2018-08-29 11:41:53,719 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Created
directory /opt/flink/upload/flink-web-upload for file uploads.
2018-08-29 11:41:53,722 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting
rest endpoint.
2018-08-29 11:41:54,084 WARN
org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file
environment variable 'log.file' is not set.
2018-08-29 11:41:54,084 WARN
org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager
log files are unavailable in the web dashboard. Log file location not found
in environment variable 'log.file' or configuration key 'Key:
'web.log.path' , default: null (deprecated keys:
[jobmanager.web.log.path])'.
2018-08-29 11:41:54,160 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest
endpoint listening at flink-jobmanager-2:8081
2018-08-29 11:41:54,160 INFO
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
Starting ZooKeeperLeaderElectionService
ZooKeeperLeaderElectionService{leaderPath='/leader/rest_server_lock'}.
2018-08-29 11:41:54,180 INFO
org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Web
frontend listening at http://flink-jobmanager-2:8081.
2018-08-29 11:41:54,192 INFO
org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
RPC endpoint for
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at
akka://flink/user/resourcemanager .
2018-08-29 11:41:54,273 INFO
org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting
RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher
at akka://flink/user/dispatcher .
2018-08-29 11:41:54,286 INFO
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
Starting ZooKeeperLeaderElectionService
ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
2018-08-29 11:41:54,287 INFO
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2018-08-29 11:41:54,289 INFO
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
Starting ZooKeeperLeaderElectionService
ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
2018-08-29 11:41:54,289 INFO
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
Starting ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.

*Upon submitting a batch job on Jobmanager-1, we immediately get this log
on Jobmanager-2*
2018-08-29 11:47:06,249 INFO
org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
Recovered SubmittedJobGraph(d69b67e4d28a2d244b06d3f6d661bca1, null).

*Meanwhile Jobmanager-1 gets:*
*-FlinkBatchPipelineTranslator pipeline logs- (we use Apache Beam)*

2018-08-29 11:47:06,006 INFO
org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Submitting
job d69b67e4d28a2d244b06d3f6d661bca1
(sicassandrawriterbeam-flink-0829114703-7d95fabd).
2018-08-29 11:47:06,090 INFO
org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
Added SubmittedJobGraph(d69b67e4d28a2d244b06d3f6d661bca1, null) to
ZooKeeper.

*-loads of job execution info-*

2018-08-29 11:49:20,272 INFO
org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Job
d69b67e4d28a2d244b06d3f6d661bca1 reached globally terminal state FINISHED.
2018-08-29 11:49:20,286 INFO  org.apache.flink.runtime.jobmaster.JobMaster
                - Stopping the JobMaster for job
sicassandrawriterbeam-flink-0829114703-7d95fabd(d69b67e4d28a2d244b06d3f6d661bca1).
2018-08-29 11:49:20,290 INFO
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
Stopping ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2018-08-29 11:49:20,292 INFO  org.apache.flink.runtime.jobmaster.JobMaster
                - Close ResourceManager connection
827b94881bf7c94d8516907e04e3a564: JobManager is shutting down..
2018-08-29 11:49:20,292 INFO
org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Suspending
SlotPool.
2018-08-29 11:49:20,293 INFO
org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Stopping
SlotPool.
2018-08-29 11:49:20,293 INFO
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  -
Disconnect job manager a3dab0a0883c5f0f37943358d9104d79
@akka.tcp://flink@flink-jobmanager-1:50010/user/jobmanager_0 for job
d69b67e4d28a2d244b06d3f6d661bca1 from the resource manager.
2018-08-29 11:49:20,293 INFO
org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  -
Stopping ZooKeeperLeaderElectionService
ZooKeeperLeaderElectionService{leaderPath='/leader/d69b67e4d28a2d244b06d3f6d661bca1/job_manager_lock'}.
2018-08-29 11:49:20,304 INFO
org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
Removed job graph d69b67e4d28a2d244b06d3f6d661bca1 from ZooKeeper.


-------------------

The result is:
HDFS has only a jobgraph and an empty default folder - everything else is
cleared
ZooKeeper has the jobgraph that Jobmanager-1 claims to have removed in the
last log still there.

On Wed, Aug 29, 2018 at 12:14 PM Till Rohrmann <tr...@apache.org> wrote:

> Hi Encho,
>
> it sounds strange that the standby JobManager tries to recover a submitted
> job graph. This should only happen if it has been granted leadership. Thus,
> it seems as if the standby JobManager thinks that it is also the leader.
> Could you maybe share the logs of the two JobManagers/ClusterEntrypoints
> with us?
>
> Running only a single JobManager/ClusterEntrypoint in HA mode via a
> Kubernetes Deployment should do the trick and there is nothing wrong with
> it.
>
> Cheers,
> Till
>
> On Wed, Aug 29, 2018 at 11:05 AM Encho Mishinev <en...@gmail.com>
> wrote:
>
>> Hello,
>>
>> Since two job managers don't seem to be working for me I was thinking of
>> just using a single job manager in Kubernetes in HA mode with a deployment
>> ensuring its restart whenever it fails. Is this approach viable? The
>> High-Availability page mentions that you use only one job manager in an
>> YARN cluster but does not specify such option for Kubernetes. Is there
>> anything that can go wrong with this approach?
>>
>> Thanks
>>
>> On Wed, Aug 29, 2018 at 11:10 AM Encho Mishinev <en...@gmail.com>
>> wrote:
>>
>>> Hi,
>>>
>>> Unfortunately the thing I described does indeed happen every time. As
>>> mentioned in the first email, I am running on Kubernetes so certain things
>>> could be different compared to just a standalone cluster.
>>>
>>> Any ideas for workarounds are welcome, as this problem basically
>>> prevents me from using HA.
>>>
>>> Thanks,
>>> Encho
>>>
>>> On Wed, Aug 29, 2018 at 5:15 AM vino yang <ya...@gmail.com> wrote:
>>>
>>>> Hi Encho,
>>>>
>>>> From your description, I feel that there are extra bugs.
>>>>
>>>> About your description:
>>>>
>>>> *- Start both job managers*
>>>> *- Start a batch job in JobManager 1 and let it finish*
>>>> *The jobgraphs in both Zookeeper and HDFS remained.*
>>>>
>>>> Is it necessarily happening every time?
>>>>
>>>> In the Standalone cluster, the problems we encountered were sporadic.
>>>>
>>>> Thanks, vino.
>>>>
>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午8:07写道:
>>>>
>>>>> Hello Till,
>>>>>
>>>>> I spend a few more hours testing and looking at the logs and it seems
>>>>> like there's a more general problem here. While the two job managers are
>>>>> active neither of them can properly delete jobgraphs. The above problem I
>>>>> described comes from the fact that Kubernetes gets JobManager 1 quickly
>>>>> after I manually kill it, so when I stop the job on JobManager 2 both are
>>>>> alive.
>>>>>
>>>>> I did a very simple test:
>>>>>
>>>>> - Start both job managers
>>>>> - Start a batch job in JobManager 1 and let it finish
>>>>> The jobgraphs in both Zookeeper and HDFS remained.
>>>>>
>>>>> On the other hand if we do:
>>>>>
>>>>> - Start only JobManager 1 (again in HA mode)
>>>>> - Start a batch job and let it finish
>>>>> The jobgraphs in both Zookeeper and HDFS are deleted fine.
>>>>>
>>>>> It seems like the standby manager still leaves some kind of lock on
>>>>> the jobgraphs. Do you think that's possible? Have you seen a similar
>>>>> problem?
>>>>> The only logs that appear on the standby manager while waiting are of
>>>>> the type:
>>>>>
>>>>> 2018-08-28 11:54:10,789 INFO
>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>> Recovered SubmittedJobGraph(9e0a109b57511930c95d3b54574a66e3, null).
>>>>>
>>>>> Note that this log appears on the standby jobmanager immediately when
>>>>> a new job is submitted to the active jobmanager.
>>>>> Also note that the blobs and checkpoints are cleared fine. The problem
>>>>> is only for jobgraphs both in ZooKeeper and HDFS.
>>>>>
>>>>> Trying to access the UI of the standby manager redirects to the active
>>>>> one, so it is not a problem of them not knowing who the leader is. Do you
>>>>> have any ideas?
>>>>>
>>>>> Thanks a lot,
>>>>> Encho
>>>>>
>>>>> On Tue, Aug 28, 2018 at 10:27 AM Till Rohrmann <tr...@apache.org>
>>>>> wrote:
>>>>>
>>>>>> Hi Encho,
>>>>>>
>>>>>> thanks a lot for reporting this issue. The problem arises whenever
>>>>>> the old leader maintains the connection to ZooKeeper. If this is the case,
>>>>>> then ephemeral nodes which we create to protect against faulty delete
>>>>>> operations are not removed and consequently the new leader is not able to
>>>>>> delete the persisted job graph. So one thing to check is whether the old JM
>>>>>> still has an open connection to ZooKeeper. The next thing to check is the
>>>>>> session timeout of your ZooKeeper cluster. If you stop the job within the
>>>>>> session timeout, then it is also not guaranteed that ZooKeeper has detected
>>>>>> that the ephemeral nodes of the old JM must be deleted. In order to
>>>>>> understand this better it would be helpful if you could tell us the timing
>>>>>> of the different actions.
>>>>>>
>>>>>> Cheers,
>>>>>> Till
>>>>>>
>>>>>> On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi Encho,
>>>>>>>
>>>>>>> A temporary solution can be used to determine if it has been cleaned
>>>>>>> up by monitoring the specific JobID under Zookeeper's "/jobgraph".
>>>>>>> Another solution, modify the source code, rudely modify the cleanup
>>>>>>> mode to the synchronous form, but the flink operation Zookeeper's path
>>>>>>> needs to obtain the corresponding lock, so it is dangerous to do so, and it
>>>>>>> is not recommended.
>>>>>>> I think maybe this problem can be solved in the next version. It
>>>>>>> depends on Till.
>>>>>>>
>>>>>>> Thanks, vino.
>>>>>>>
>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:
>>>>>>>
>>>>>>>> Thank you very much for the info! Will keep track of the progress.
>>>>>>>>
>>>>>>>> In the meantime is there any viable workaround? It seems like HA
>>>>>>>> doesn't really work due to this bug.
>>>>>>>>
>>>>>>>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> About some implementation mechanisms.
>>>>>>>>> Flink uses Zookeeper to store JobGraph (Job's description
>>>>>>>>> information and metadata) as a basis for Job recovery.
>>>>>>>>> However, previous implementations may cause this information to
>>>>>>>>> not be properly cleaned up because it is asynchronously deleted by a
>>>>>>>>> background thread.
>>>>>>>>>
>>>>>>>>> Thanks, vino.
>>>>>>>>>
>>>>>>>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>>>>>>>
>>>>>>>>>> Hi Encho,
>>>>>>>>>>
>>>>>>>>>> This is a problem already known to the Flink community, you can
>>>>>>>>>> track its progress through FLINK-10011[1], and currently Till is fixing
>>>>>>>>>> this issue.
>>>>>>>>>>
>>>>>>>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>>>>>>>
>>>>>>>>>> Thanks, vino.
>>>>>>>>>>
>>>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一
>>>>>>>>>> 下午10:13写道:
>>>>>>>>>>
>>>>>>>>>>> I am running Flink 1.5.3 with two job managers and two task
>>>>>>>>>>> managers in Kubernetes along with HDFS and Zookeeper in high-availability
>>>>>>>>>>> mode.
>>>>>>>>>>>
>>>>>>>>>>> My problem occurs after the following actions:
>>>>>>>>>>> - Upload a .jar file to jobmanager-1
>>>>>>>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>>>>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>>>>>>>> - Kill pod of jobmanager-1
>>>>>>>>>>> After a short delay, jobmanager-2 takes leadership and correctly
>>>>>>>>>>> restores the job and continues it
>>>>>>>>>>> - Stop job from jobmanager-2
>>>>>>>>>>>
>>>>>>>>>>> At this point all seems well, but the problem is that
>>>>>>>>>>> jobmanager-2 does not clean up anything that was left from jobmanager-1.
>>>>>>>>>>> This means that both in HDFS and in Zookeeper remain job graphs, which
>>>>>>>>>>> later on obstruct any work of both managers as after any reset they
>>>>>>>>>>> unsuccessfully try to restore a non-existent job and fail over and over
>>>>>>>>>>> again.
>>>>>>>>>>>
>>>>>>>>>>> I am quite certain that jobmanager-2 does not know about any of
>>>>>>>>>>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>>>>>>>> duplicate job folders:
>>>>>>>>>>>
>>>>>>>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>>
>>>>>>>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>>>>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>
>>>>>>>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in
>>>>>>>>>>> Zookeeper when the job is stopped, but fails since there are leftover files
>>>>>>>>>>> in it from jobmanager-1:
>>>>>>>>>>>
>>>>>>>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>>>>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>>
>>>>>>>>>>> I’ve noticed that when restoring the job, it seems like
>>>>>>>>>>> jobmanager-2 does not get anything more than jobID, while it perhaps needs
>>>>>>>>>>> some metadata? Here is the log that seems suspicious to me:
>>>>>>>>>>>
>>>>>>>>>>> 2018-08-27 13:09:18,113 INFO
>>>>>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>>>>>>>
>>>>>>>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to be
>>>>>>>>>>> aware that it’s overwriting anything or not deleting properly.
>>>>>>>>>>>
>>>>>>>>>>> My question is - what is the intended way for the job managers
>>>>>>>>>>> to correctly exchange metadata in HA mode and why is it not working for me?
>>>>>>>>>>>
>>>>>>>>>>> Thanks in advance!
>>>>>>>>>>
>>>>>>>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by Till Rohrmann <tr...@apache.org>.
Hi Encho,

it sounds strange that the standby JobManager tries to recover a submitted
job graph. This should only happen if it has been granted leadership. Thus,
it seems as if the standby JobManager thinks that it is also the leader.
Could you maybe share the logs of the two JobManagers/ClusterEntrypoints
with us?

Running only a single JobManager/ClusterEntrypoint in HA mode via a
Kubernetes Deployment should do the trick and there is nothing wrong with
it.

Cheers,
Till

On Wed, Aug 29, 2018 at 11:05 AM Encho Mishinev <en...@gmail.com>
wrote:

> Hello,
>
> Since two job managers don't seem to be working for me I was thinking of
> just using a single job manager in Kubernetes in HA mode with a deployment
> ensuring its restart whenever it fails. Is this approach viable? The
> High-Availability page mentions that you use only one job manager in an
> YARN cluster but does not specify such option for Kubernetes. Is there
> anything that can go wrong with this approach?
>
> Thanks
>
> On Wed, Aug 29, 2018 at 11:10 AM Encho Mishinev <en...@gmail.com>
> wrote:
>
>> Hi,
>>
>> Unfortunately the thing I described does indeed happen every time. As
>> mentioned in the first email, I am running on Kubernetes so certain things
>> could be different compared to just a standalone cluster.
>>
>> Any ideas for workarounds are welcome, as this problem basically prevents
>> me from using HA.
>>
>> Thanks,
>> Encho
>>
>> On Wed, Aug 29, 2018 at 5:15 AM vino yang <ya...@gmail.com> wrote:
>>
>>> Hi Encho,
>>>
>>> From your description, I feel that there are extra bugs.
>>>
>>> About your description:
>>>
>>> *- Start both job managers*
>>> *- Start a batch job in JobManager 1 and let it finish*
>>> *The jobgraphs in both Zookeeper and HDFS remained.*
>>>
>>> Is it necessarily happening every time?
>>>
>>> In the Standalone cluster, the problems we encountered were sporadic.
>>>
>>> Thanks, vino.
>>>
>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午8:07写道:
>>>
>>>> Hello Till,
>>>>
>>>> I spend a few more hours testing and looking at the logs and it seems
>>>> like there's a more general problem here. While the two job managers are
>>>> active neither of them can properly delete jobgraphs. The above problem I
>>>> described comes from the fact that Kubernetes gets JobManager 1 quickly
>>>> after I manually kill it, so when I stop the job on JobManager 2 both are
>>>> alive.
>>>>
>>>> I did a very simple test:
>>>>
>>>> - Start both job managers
>>>> - Start a batch job in JobManager 1 and let it finish
>>>> The jobgraphs in both Zookeeper and HDFS remained.
>>>>
>>>> On the other hand if we do:
>>>>
>>>> - Start only JobManager 1 (again in HA mode)
>>>> - Start a batch job and let it finish
>>>> The jobgraphs in both Zookeeper and HDFS are deleted fine.
>>>>
>>>> It seems like the standby manager still leaves some kind of lock on the
>>>> jobgraphs. Do you think that's possible? Have you seen a similar problem?
>>>> The only logs that appear on the standby manager while waiting are of
>>>> the type:
>>>>
>>>> 2018-08-28 11:54:10,789 INFO
>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>> Recovered SubmittedJobGraph(9e0a109b57511930c95d3b54574a66e3, null).
>>>>
>>>> Note that this log appears on the standby jobmanager immediately when a
>>>> new job is submitted to the active jobmanager.
>>>> Also note that the blobs and checkpoints are cleared fine. The problem
>>>> is only for jobgraphs both in ZooKeeper and HDFS.
>>>>
>>>> Trying to access the UI of the standby manager redirects to the active
>>>> one, so it is not a problem of them not knowing who the leader is. Do you
>>>> have any ideas?
>>>>
>>>> Thanks a lot,
>>>> Encho
>>>>
>>>> On Tue, Aug 28, 2018 at 10:27 AM Till Rohrmann <tr...@apache.org>
>>>> wrote:
>>>>
>>>>> Hi Encho,
>>>>>
>>>>> thanks a lot for reporting this issue. The problem arises whenever the
>>>>> old leader maintains the connection to ZooKeeper. If this is the case, then
>>>>> ephemeral nodes which we create to protect against faulty delete operations
>>>>> are not removed and consequently the new leader is not able to delete the
>>>>> persisted job graph. So one thing to check is whether the old JM still has
>>>>> an open connection to ZooKeeper. The next thing to check is the session
>>>>> timeout of your ZooKeeper cluster. If you stop the job within the session
>>>>> timeout, then it is also not guaranteed that ZooKeeper has detected that
>>>>> the ephemeral nodes of the old JM must be deleted. In order to understand
>>>>> this better it would be helpful if you could tell us the timing of the
>>>>> different actions.
>>>>>
>>>>> Cheers,
>>>>> Till
>>>>>
>>>>> On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi Encho,
>>>>>>
>>>>>> A temporary solution can be used to determine if it has been cleaned
>>>>>> up by monitoring the specific JobID under Zookeeper's "/jobgraph".
>>>>>> Another solution, modify the source code, rudely modify the cleanup
>>>>>> mode to the synchronous form, but the flink operation Zookeeper's path
>>>>>> needs to obtain the corresponding lock, so it is dangerous to do so, and it
>>>>>> is not recommended.
>>>>>> I think maybe this problem can be solved in the next version. It
>>>>>> depends on Till.
>>>>>>
>>>>>> Thanks, vino.
>>>>>>
>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:
>>>>>>
>>>>>>> Thank you very much for the info! Will keep track of the progress.
>>>>>>>
>>>>>>> In the meantime is there any viable workaround? It seems like HA
>>>>>>> doesn't really work due to this bug.
>>>>>>>
>>>>>>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> About some implementation mechanisms.
>>>>>>>> Flink uses Zookeeper to store JobGraph (Job's description
>>>>>>>> information and metadata) as a basis for Job recovery.
>>>>>>>> However, previous implementations may cause this information to not
>>>>>>>> be properly cleaned up because it is asynchronously deleted by a background
>>>>>>>> thread.
>>>>>>>>
>>>>>>>> Thanks, vino.
>>>>>>>>
>>>>>>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>>>>>>
>>>>>>>>> Hi Encho,
>>>>>>>>>
>>>>>>>>> This is a problem already known to the Flink community, you can
>>>>>>>>> track its progress through FLINK-10011[1], and currently Till is fixing
>>>>>>>>> this issue.
>>>>>>>>>
>>>>>>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>>>>>>
>>>>>>>>> Thanks, vino.
>>>>>>>>>
>>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一 下午10:13写道:
>>>>>>>>>
>>>>>>>>>> I am running Flink 1.5.3 with two job managers and two task
>>>>>>>>>> managers in Kubernetes along with HDFS and Zookeeper in high-availability
>>>>>>>>>> mode.
>>>>>>>>>>
>>>>>>>>>> My problem occurs after the following actions:
>>>>>>>>>> - Upload a .jar file to jobmanager-1
>>>>>>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>>>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>>>>>>> - Kill pod of jobmanager-1
>>>>>>>>>> After a short delay, jobmanager-2 takes leadership and correctly
>>>>>>>>>> restores the job and continues it
>>>>>>>>>> - Stop job from jobmanager-2
>>>>>>>>>>
>>>>>>>>>> At this point all seems well, but the problem is that
>>>>>>>>>> jobmanager-2 does not clean up anything that was left from jobmanager-1.
>>>>>>>>>> This means that both in HDFS and in Zookeeper remain job graphs, which
>>>>>>>>>> later on obstruct any work of both managers as after any reset they
>>>>>>>>>> unsuccessfully try to restore a non-existent job and fail over and over
>>>>>>>>>> again.
>>>>>>>>>>
>>>>>>>>>> I am quite certain that jobmanager-2 does not know about any of
>>>>>>>>>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>>>>>>> duplicate job folders:
>>>>>>>>>>
>>>>>>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>>
>>>>>>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>>>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>
>>>>>>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in
>>>>>>>>>> Zookeeper when the job is stopped, but fails since there are leftover files
>>>>>>>>>> in it from jobmanager-1:
>>>>>>>>>>
>>>>>>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>>>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>>
>>>>>>>>>> I’ve noticed that when restoring the job, it seems like
>>>>>>>>>> jobmanager-2 does not get anything more than jobID, while it perhaps needs
>>>>>>>>>> some metadata? Here is the log that seems suspicious to me:
>>>>>>>>>>
>>>>>>>>>> 2018-08-27 13:09:18,113 INFO
>>>>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>>>>>>
>>>>>>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to be
>>>>>>>>>> aware that it’s overwriting anything or not deleting properly.
>>>>>>>>>>
>>>>>>>>>> My question is - what is the intended way for the job managers to
>>>>>>>>>> correctly exchange metadata in HA mode and why is it not working for me?
>>>>>>>>>>
>>>>>>>>>> Thanks in advance!
>>>>>>>>>
>>>>>>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by Encho Mishinev <en...@gmail.com>.
Hello,

Since two job managers don't seem to be working for me I was thinking of
just using a single job manager in Kubernetes in HA mode with a deployment
ensuring its restart whenever it fails. Is this approach viable? The
High-Availability page mentions that you use only one job manager in an
YARN cluster but does not specify such option for Kubernetes. Is there
anything that can go wrong with this approach?

Thanks

On Wed, Aug 29, 2018 at 11:10 AM Encho Mishinev <en...@gmail.com>
wrote:

> Hi,
>
> Unfortunately the thing I described does indeed happen every time. As
> mentioned in the first email, I am running on Kubernetes so certain things
> could be different compared to just a standalone cluster.
>
> Any ideas for workarounds are welcome, as this problem basically prevents
> me from using HA.
>
> Thanks,
> Encho
>
> On Wed, Aug 29, 2018 at 5:15 AM vino yang <ya...@gmail.com> wrote:
>
>> Hi Encho,
>>
>> From your description, I feel that there are extra bugs.
>>
>> About your description:
>>
>> *- Start both job managers*
>> *- Start a batch job in JobManager 1 and let it finish*
>> *The jobgraphs in both Zookeeper and HDFS remained.*
>>
>> Is it necessarily happening every time?
>>
>> In the Standalone cluster, the problems we encountered were sporadic.
>>
>> Thanks, vino.
>>
>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午8:07写道:
>>
>>> Hello Till,
>>>
>>> I spend a few more hours testing and looking at the logs and it seems
>>> like there's a more general problem here. While the two job managers are
>>> active neither of them can properly delete jobgraphs. The above problem I
>>> described comes from the fact that Kubernetes gets JobManager 1 quickly
>>> after I manually kill it, so when I stop the job on JobManager 2 both are
>>> alive.
>>>
>>> I did a very simple test:
>>>
>>> - Start both job managers
>>> - Start a batch job in JobManager 1 and let it finish
>>> The jobgraphs in both Zookeeper and HDFS remained.
>>>
>>> On the other hand if we do:
>>>
>>> - Start only JobManager 1 (again in HA mode)
>>> - Start a batch job and let it finish
>>> The jobgraphs in both Zookeeper and HDFS are deleted fine.
>>>
>>> It seems like the standby manager still leaves some kind of lock on the
>>> jobgraphs. Do you think that's possible? Have you seen a similar problem?
>>> The only logs that appear on the standby manager while waiting are of
>>> the type:
>>>
>>> 2018-08-28 11:54:10,789 INFO
>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>> Recovered SubmittedJobGraph(9e0a109b57511930c95d3b54574a66e3, null).
>>>
>>> Note that this log appears on the standby jobmanager immediately when a
>>> new job is submitted to the active jobmanager.
>>> Also note that the blobs and checkpoints are cleared fine. The problem
>>> is only for jobgraphs both in ZooKeeper and HDFS.
>>>
>>> Trying to access the UI of the standby manager redirects to the active
>>> one, so it is not a problem of them not knowing who the leader is. Do you
>>> have any ideas?
>>>
>>> Thanks a lot,
>>> Encho
>>>
>>> On Tue, Aug 28, 2018 at 10:27 AM Till Rohrmann <tr...@apache.org>
>>> wrote:
>>>
>>>> Hi Encho,
>>>>
>>>> thanks a lot for reporting this issue. The problem arises whenever the
>>>> old leader maintains the connection to ZooKeeper. If this is the case, then
>>>> ephemeral nodes which we create to protect against faulty delete operations
>>>> are not removed and consequently the new leader is not able to delete the
>>>> persisted job graph. So one thing to check is whether the old JM still has
>>>> an open connection to ZooKeeper. The next thing to check is the session
>>>> timeout of your ZooKeeper cluster. If you stop the job within the session
>>>> timeout, then it is also not guaranteed that ZooKeeper has detected that
>>>> the ephemeral nodes of the old JM must be deleted. In order to understand
>>>> this better it would be helpful if you could tell us the timing of the
>>>> different actions.
>>>>
>>>> Cheers,
>>>> Till
>>>>
>>>> On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi Encho,
>>>>>
>>>>> A temporary solution can be used to determine if it has been cleaned
>>>>> up by monitoring the specific JobID under Zookeeper's "/jobgraph".
>>>>> Another solution, modify the source code, rudely modify the cleanup
>>>>> mode to the synchronous form, but the flink operation Zookeeper's path
>>>>> needs to obtain the corresponding lock, so it is dangerous to do so, and it
>>>>> is not recommended.
>>>>> I think maybe this problem can be solved in the next version. It
>>>>> depends on Till.
>>>>>
>>>>> Thanks, vino.
>>>>>
>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:
>>>>>
>>>>>> Thank you very much for the info! Will keep track of the progress.
>>>>>>
>>>>>> In the meantime is there any viable workaround? It seems like HA
>>>>>> doesn't really work due to this bug.
>>>>>>
>>>>>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> About some implementation mechanisms.
>>>>>>> Flink uses Zookeeper to store JobGraph (Job's description
>>>>>>> information and metadata) as a basis for Job recovery.
>>>>>>> However, previous implementations may cause this information to not
>>>>>>> be properly cleaned up because it is asynchronously deleted by a background
>>>>>>> thread.
>>>>>>>
>>>>>>> Thanks, vino.
>>>>>>>
>>>>>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>>>>>
>>>>>>>> Hi Encho,
>>>>>>>>
>>>>>>>> This is a problem already known to the Flink community, you can
>>>>>>>> track its progress through FLINK-10011[1], and currently Till is fixing
>>>>>>>> this issue.
>>>>>>>>
>>>>>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>>>>>
>>>>>>>> Thanks, vino.
>>>>>>>>
>>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一 下午10:13写道:
>>>>>>>>
>>>>>>>>> I am running Flink 1.5.3 with two job managers and two task
>>>>>>>>> managers in Kubernetes along with HDFS and Zookeeper in high-availability
>>>>>>>>> mode.
>>>>>>>>>
>>>>>>>>> My problem occurs after the following actions:
>>>>>>>>> - Upload a .jar file to jobmanager-1
>>>>>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>>>>>> - Kill pod of jobmanager-1
>>>>>>>>> After a short delay, jobmanager-2 takes leadership and correctly
>>>>>>>>> restores the job and continues it
>>>>>>>>> - Stop job from jobmanager-2
>>>>>>>>>
>>>>>>>>> At this point all seems well, but the problem is that jobmanager-2
>>>>>>>>> does not clean up anything that was left from jobmanager-1. This means that
>>>>>>>>> both in HDFS and in Zookeeper remain job graphs, which later on obstruct
>>>>>>>>> any work of both managers as after any reset they unsuccessfully try to
>>>>>>>>> restore a non-existent job and fail over and over again.
>>>>>>>>>
>>>>>>>>> I am quite certain that jobmanager-2 does not know about any of
>>>>>>>>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>>>>>> duplicate job folders:
>>>>>>>>>
>>>>>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>>
>>>>>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>
>>>>>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in
>>>>>>>>> Zookeeper when the job is stopped, but fails since there are leftover files
>>>>>>>>> in it from jobmanager-1:
>>>>>>>>>
>>>>>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>>
>>>>>>>>> I’ve noticed that when restoring the job, it seems like
>>>>>>>>> jobmanager-2 does not get anything more than jobID, while it perhaps needs
>>>>>>>>> some metadata? Here is the log that seems suspicious to me:
>>>>>>>>>
>>>>>>>>> 2018-08-27 13:09:18,113 INFO
>>>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>>>>>
>>>>>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to be
>>>>>>>>> aware that it’s overwriting anything or not deleting properly.
>>>>>>>>>
>>>>>>>>> My question is - what is the intended way for the job managers to
>>>>>>>>> correctly exchange metadata in HA mode and why is it not working for me?
>>>>>>>>>
>>>>>>>>> Thanks in advance!
>>>>>>>>
>>>>>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by Encho Mishinev <en...@gmail.com>.
Hi,

Unfortunately the thing I described does indeed happen every time. As
mentioned in the first email, I am running on Kubernetes so certain things
could be different compared to just a standalone cluster.

Any ideas for workarounds are welcome, as this problem basically prevents
me from using HA.

Thanks,
Encho

On Wed, Aug 29, 2018 at 5:15 AM vino yang <ya...@gmail.com> wrote:

> Hi Encho,
>
> From your description, I feel that there are extra bugs.
>
> About your description:
>
> *- Start both job managers*
> *- Start a batch job in JobManager 1 and let it finish*
> *The jobgraphs in both Zookeeper and HDFS remained.*
>
> Is it necessarily happening every time?
>
> In the Standalone cluster, the problems we encountered were sporadic.
>
> Thanks, vino.
>
> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午8:07写道:
>
>> Hello Till,
>>
>> I spend a few more hours testing and looking at the logs and it seems
>> like there's a more general problem here. While the two job managers are
>> active neither of them can properly delete jobgraphs. The above problem I
>> described comes from the fact that Kubernetes gets JobManager 1 quickly
>> after I manually kill it, so when I stop the job on JobManager 2 both are
>> alive.
>>
>> I did a very simple test:
>>
>> - Start both job managers
>> - Start a batch job in JobManager 1 and let it finish
>> The jobgraphs in both Zookeeper and HDFS remained.
>>
>> On the other hand if we do:
>>
>> - Start only JobManager 1 (again in HA mode)
>> - Start a batch job and let it finish
>> The jobgraphs in both Zookeeper and HDFS are deleted fine.
>>
>> It seems like the standby manager still leaves some kind of lock on the
>> jobgraphs. Do you think that's possible? Have you seen a similar problem?
>> The only logs that appear on the standby manager while waiting are of the
>> type:
>>
>> 2018-08-28 11:54:10,789 INFO
>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>> Recovered SubmittedJobGraph(9e0a109b57511930c95d3b54574a66e3, null).
>>
>> Note that this log appears on the standby jobmanager immediately when a
>> new job is submitted to the active jobmanager.
>> Also note that the blobs and checkpoints are cleared fine. The problem is
>> only for jobgraphs both in ZooKeeper and HDFS.
>>
>> Trying to access the UI of the standby manager redirects to the active
>> one, so it is not a problem of them not knowing who the leader is. Do you
>> have any ideas?
>>
>> Thanks a lot,
>> Encho
>>
>> On Tue, Aug 28, 2018 at 10:27 AM Till Rohrmann <tr...@apache.org>
>> wrote:
>>
>>> Hi Encho,
>>>
>>> thanks a lot for reporting this issue. The problem arises whenever the
>>> old leader maintains the connection to ZooKeeper. If this is the case, then
>>> ephemeral nodes which we create to protect against faulty delete operations
>>> are not removed and consequently the new leader is not able to delete the
>>> persisted job graph. So one thing to check is whether the old JM still has
>>> an open connection to ZooKeeper. The next thing to check is the session
>>> timeout of your ZooKeeper cluster. If you stop the job within the session
>>> timeout, then it is also not guaranteed that ZooKeeper has detected that
>>> the ephemeral nodes of the old JM must be deleted. In order to understand
>>> this better it would be helpful if you could tell us the timing of the
>>> different actions.
>>>
>>> Cheers,
>>> Till
>>>
>>> On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com> wrote:
>>>
>>>> Hi Encho,
>>>>
>>>> A temporary solution can be used to determine if it has been cleaned up
>>>> by monitoring the specific JobID under Zookeeper's "/jobgraph".
>>>> Another solution, modify the source code, rudely modify the cleanup
>>>> mode to the synchronous form, but the flink operation Zookeeper's path
>>>> needs to obtain the corresponding lock, so it is dangerous to do so, and it
>>>> is not recommended.
>>>> I think maybe this problem can be solved in the next version. It
>>>> depends on Till.
>>>>
>>>> Thanks, vino.
>>>>
>>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:
>>>>
>>>>> Thank you very much for the info! Will keep track of the progress.
>>>>>
>>>>> In the meantime is there any viable workaround? It seems like HA
>>>>> doesn't really work due to this bug.
>>>>>
>>>>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> About some implementation mechanisms.
>>>>>> Flink uses Zookeeper to store JobGraph (Job's description information
>>>>>> and metadata) as a basis for Job recovery.
>>>>>> However, previous implementations may cause this information to not
>>>>>> be properly cleaned up because it is asynchronously deleted by a background
>>>>>> thread.
>>>>>>
>>>>>> Thanks, vino.
>>>>>>
>>>>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>>>>
>>>>>>> Hi Encho,
>>>>>>>
>>>>>>> This is a problem already known to the Flink community, you can
>>>>>>> track its progress through FLINK-10011[1], and currently Till is fixing
>>>>>>> this issue.
>>>>>>>
>>>>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>>>>
>>>>>>> Thanks, vino.
>>>>>>>
>>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一 下午10:13写道:
>>>>>>>
>>>>>>>> I am running Flink 1.5.3 with two job managers and two task
>>>>>>>> managers in Kubernetes along with HDFS and Zookeeper in high-availability
>>>>>>>> mode.
>>>>>>>>
>>>>>>>> My problem occurs after the following actions:
>>>>>>>> - Upload a .jar file to jobmanager-1
>>>>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>>>>> - Kill pod of jobmanager-1
>>>>>>>> After a short delay, jobmanager-2 takes leadership and correctly
>>>>>>>> restores the job and continues it
>>>>>>>> - Stop job from jobmanager-2
>>>>>>>>
>>>>>>>> At this point all seems well, but the problem is that jobmanager-2
>>>>>>>> does not clean up anything that was left from jobmanager-1. This means that
>>>>>>>> both in HDFS and in Zookeeper remain job graphs, which later on obstruct
>>>>>>>> any work of both managers as after any reset they unsuccessfully try to
>>>>>>>> restore a non-existent job and fail over and over again.
>>>>>>>>
>>>>>>>> I am quite certain that jobmanager-2 does not know about any of
>>>>>>>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>>>>> duplicate job folders:
>>>>>>>>
>>>>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>>
>>>>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>
>>>>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in
>>>>>>>> Zookeeper when the job is stopped, but fails since there are leftover files
>>>>>>>> in it from jobmanager-1:
>>>>>>>>
>>>>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>>
>>>>>>>> I’ve noticed that when restoring the job, it seems like
>>>>>>>> jobmanager-2 does not get anything more than jobID, while it perhaps needs
>>>>>>>> some metadata? Here is the log that seems suspicious to me:
>>>>>>>>
>>>>>>>> 2018-08-27 13:09:18,113 INFO
>>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>>>>
>>>>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to be
>>>>>>>> aware that it’s overwriting anything or not deleting properly.
>>>>>>>>
>>>>>>>> My question is - what is the intended way for the job managers to
>>>>>>>> correctly exchange metadata in HA mode and why is it not working for me?
>>>>>>>>
>>>>>>>> Thanks in advance!
>>>>>>>
>>>>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by vino yang <ya...@gmail.com>.
Hi Encho,

From your description, I feel that there are extra bugs.

About your description:

*- Start both job managers*
*- Start a batch job in JobManager 1 and let it finish*
*The jobgraphs in both Zookeeper and HDFS remained.*

Is it necessarily happening every time?

In the Standalone cluster, the problems we encountered were sporadic.

Thanks, vino.

Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午8:07写道:

> Hello Till,
>
> I spend a few more hours testing and looking at the logs and it seems like
> there's a more general problem here. While the two job managers are active
> neither of them can properly delete jobgraphs. The above problem I
> described comes from the fact that Kubernetes gets JobManager 1 quickly
> after I manually kill it, so when I stop the job on JobManager 2 both are
> alive.
>
> I did a very simple test:
>
> - Start both job managers
> - Start a batch job in JobManager 1 and let it finish
> The jobgraphs in both Zookeeper and HDFS remained.
>
> On the other hand if we do:
>
> - Start only JobManager 1 (again in HA mode)
> - Start a batch job and let it finish
> The jobgraphs in both Zookeeper and HDFS are deleted fine.
>
> It seems like the standby manager still leaves some kind of lock on the
> jobgraphs. Do you think that's possible? Have you seen a similar problem?
> The only logs that appear on the standby manager while waiting are of the
> type:
>
> 2018-08-28 11:54:10,789 INFO
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
> Recovered SubmittedJobGraph(9e0a109b57511930c95d3b54574a66e3, null).
>
> Note that this log appears on the standby jobmanager immediately when a
> new job is submitted to the active jobmanager.
> Also note that the blobs and checkpoints are cleared fine. The problem is
> only for jobgraphs both in ZooKeeper and HDFS.
>
> Trying to access the UI of the standby manager redirects to the active
> one, so it is not a problem of them not knowing who the leader is. Do you
> have any ideas?
>
> Thanks a lot,
> Encho
>
> On Tue, Aug 28, 2018 at 10:27 AM Till Rohrmann <tr...@apache.org>
> wrote:
>
>> Hi Encho,
>>
>> thanks a lot for reporting this issue. The problem arises whenever the
>> old leader maintains the connection to ZooKeeper. If this is the case, then
>> ephemeral nodes which we create to protect against faulty delete operations
>> are not removed and consequently the new leader is not able to delete the
>> persisted job graph. So one thing to check is whether the old JM still has
>> an open connection to ZooKeeper. The next thing to check is the session
>> timeout of your ZooKeeper cluster. If you stop the job within the session
>> timeout, then it is also not guaranteed that ZooKeeper has detected that
>> the ephemeral nodes of the old JM must be deleted. In order to understand
>> this better it would be helpful if you could tell us the timing of the
>> different actions.
>>
>> Cheers,
>> Till
>>
>> On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com> wrote:
>>
>>> Hi Encho,
>>>
>>> A temporary solution can be used to determine if it has been cleaned up
>>> by monitoring the specific JobID under Zookeeper's "/jobgraph".
>>> Another solution, modify the source code, rudely modify the cleanup mode
>>> to the synchronous form, but the flink operation Zookeeper's path needs to
>>> obtain the corresponding lock, so it is dangerous to do so, and it is not
>>> recommended.
>>> I think maybe this problem can be solved in the next version. It depends
>>> on Till.
>>>
>>> Thanks, vino.
>>>
>>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:
>>>
>>>> Thank you very much for the info! Will keep track of the progress.
>>>>
>>>> In the meantime is there any viable workaround? It seems like HA
>>>> doesn't really work due to this bug.
>>>>
>>>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com>
>>>> wrote:
>>>>
>>>>> About some implementation mechanisms.
>>>>> Flink uses Zookeeper to store JobGraph (Job's description information
>>>>> and metadata) as a basis for Job recovery.
>>>>> However, previous implementations may cause this information to not be
>>>>> properly cleaned up because it is asynchronously deleted by a background
>>>>> thread.
>>>>>
>>>>> Thanks, vino.
>>>>>
>>>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>>>
>>>>>> Hi Encho,
>>>>>>
>>>>>> This is a problem already known to the Flink community, you can track
>>>>>> its progress through FLINK-10011[1], and currently Till is fixing this
>>>>>> issue.
>>>>>>
>>>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>>>
>>>>>> Thanks, vino.
>>>>>>
>>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一 下午10:13写道:
>>>>>>
>>>>>>> I am running Flink 1.5.3 with two job managers and two task managers
>>>>>>> in Kubernetes along with HDFS and Zookeeper in high-availability mode.
>>>>>>>
>>>>>>> My problem occurs after the following actions:
>>>>>>> - Upload a .jar file to jobmanager-1
>>>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>>>> - Kill pod of jobmanager-1
>>>>>>> After a short delay, jobmanager-2 takes leadership and correctly
>>>>>>> restores the job and continues it
>>>>>>> - Stop job from jobmanager-2
>>>>>>>
>>>>>>> At this point all seems well, but the problem is that jobmanager-2
>>>>>>> does not clean up anything that was left from jobmanager-1. This means that
>>>>>>> both in HDFS and in Zookeeper remain job graphs, which later on obstruct
>>>>>>> any work of both managers as after any reset they unsuccessfully try to
>>>>>>> restore a non-existent job and fail over and over again.
>>>>>>>
>>>>>>> I am quite certain that jobmanager-2 does not know about any of
>>>>>>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>>>> duplicate job folders:
>>>>>>>
>>>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>>
>>>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>
>>>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in
>>>>>>> Zookeeper when the job is stopped, but fails since there are leftover files
>>>>>>> in it from jobmanager-1:
>>>>>>>
>>>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>>
>>>>>>> I’ve noticed that when restoring the job, it seems like jobmanager-2
>>>>>>> does not get anything more than jobID, while it perhaps needs some
>>>>>>> metadata? Here is the log that seems suspicious to me:
>>>>>>>
>>>>>>> 2018-08-27 13:09:18,113 INFO
>>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>>>
>>>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to be
>>>>>>> aware that it’s overwriting anything or not deleting properly.
>>>>>>>
>>>>>>> My question is - what is the intended way for the job managers to
>>>>>>> correctly exchange metadata in HA mode and why is it not working for me?
>>>>>>>
>>>>>>> Thanks in advance!
>>>>>>
>>>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by Encho Mishinev <en...@gmail.com>.
Hello Till,

I spend a few more hours testing and looking at the logs and it seems like
there's a more general problem here. While the two job managers are active
neither of them can properly delete jobgraphs. The above problem I
described comes from the fact that Kubernetes gets JobManager 1 quickly
after I manually kill it, so when I stop the job on JobManager 2 both are
alive.

I did a very simple test:

- Start both job managers
- Start a batch job in JobManager 1 and let it finish
The jobgraphs in both Zookeeper and HDFS remained.

On the other hand if we do:

- Start only JobManager 1 (again in HA mode)
- Start a batch job and let it finish
The jobgraphs in both Zookeeper and HDFS are deleted fine.

It seems like the standby manager still leaves some kind of lock on the
jobgraphs. Do you think that's possible? Have you seen a similar problem?
The only logs that appear on the standby manager while waiting are of the
type:

2018-08-28 11:54:10,789 INFO
org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
Recovered SubmittedJobGraph(9e0a109b57511930c95d3b54574a66e3, null).

Note that this log appears on the standby jobmanager immediately when a new
job is submitted to the active jobmanager.
Also note that the blobs and checkpoints are cleared fine. The problem is
only for jobgraphs both in ZooKeeper and HDFS.

Trying to access the UI of the standby manager redirects to the active one,
so it is not a problem of them not knowing who the leader is. Do you have
any ideas?

Thanks a lot,
Encho

On Tue, Aug 28, 2018 at 10:27 AM Till Rohrmann <tr...@apache.org> wrote:

> Hi Encho,
>
> thanks a lot for reporting this issue. The problem arises whenever the old
> leader maintains the connection to ZooKeeper. If this is the case, then
> ephemeral nodes which we create to protect against faulty delete operations
> are not removed and consequently the new leader is not able to delete the
> persisted job graph. So one thing to check is whether the old JM still has
> an open connection to ZooKeeper. The next thing to check is the session
> timeout of your ZooKeeper cluster. If you stop the job within the session
> timeout, then it is also not guaranteed that ZooKeeper has detected that
> the ephemeral nodes of the old JM must be deleted. In order to understand
> this better it would be helpful if you could tell us the timing of the
> different actions.
>
> Cheers,
> Till
>
> On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com> wrote:
>
>> Hi Encho,
>>
>> A temporary solution can be used to determine if it has been cleaned up
>> by monitoring the specific JobID under Zookeeper's "/jobgraph".
>> Another solution, modify the source code, rudely modify the cleanup mode
>> to the synchronous form, but the flink operation Zookeeper's path needs to
>> obtain the corresponding lock, so it is dangerous to do so, and it is not
>> recommended.
>> I think maybe this problem can be solved in the next version. It depends
>> on Till.
>>
>> Thanks, vino.
>>
>> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:
>>
>>> Thank you very much for the info! Will keep track of the progress.
>>>
>>> In the meantime is there any viable workaround? It seems like HA doesn't
>>> really work due to this bug.
>>>
>>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com> wrote:
>>>
>>>> About some implementation mechanisms.
>>>> Flink uses Zookeeper to store JobGraph (Job's description information
>>>> and metadata) as a basis for Job recovery.
>>>> However, previous implementations may cause this information to not be
>>>> properly cleaned up because it is asynchronously deleted by a background
>>>> thread.
>>>>
>>>> Thanks, vino.
>>>>
>>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>>
>>>>> Hi Encho,
>>>>>
>>>>> This is a problem already known to the Flink community, you can track
>>>>> its progress through FLINK-10011[1], and currently Till is fixing this
>>>>> issue.
>>>>>
>>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>>
>>>>> Thanks, vino.
>>>>>
>>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一 下午10:13写道:
>>>>>
>>>>>> I am running Flink 1.5.3 with two job managers and two task managers
>>>>>> in Kubernetes along with HDFS and Zookeeper in high-availability mode.
>>>>>>
>>>>>> My problem occurs after the following actions:
>>>>>> - Upload a .jar file to jobmanager-1
>>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>>> - Kill pod of jobmanager-1
>>>>>> After a short delay, jobmanager-2 takes leadership and correctly
>>>>>> restores the job and continues it
>>>>>> - Stop job from jobmanager-2
>>>>>>
>>>>>> At this point all seems well, but the problem is that jobmanager-2
>>>>>> does not clean up anything that was left from jobmanager-1. This means that
>>>>>> both in HDFS and in Zookeeper remain job graphs, which later on obstruct
>>>>>> any work of both managers as after any reset they unsuccessfully try to
>>>>>> restore a non-existent job and fail over and over again.
>>>>>>
>>>>>> I am quite certain that jobmanager-2 does not know about any of
>>>>>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>>> duplicate job folders:
>>>>>>
>>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>>
>>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>> Error:KeeperErrorCode = NodeExists for
>>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>>
>>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in
>>>>>> Zookeeper when the job is stopped, but fails since there are leftover files
>>>>>> in it from jobmanager-1:
>>>>>>
>>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>>
>>>>>> I’ve noticed that when restoring the job, it seems like jobmanager-2
>>>>>> does not get anything more than jobID, while it perhaps needs some
>>>>>> metadata? Here is the log that seems suspicious to me:
>>>>>>
>>>>>> 2018-08-27 13:09:18,113 INFO
>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>>
>>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to be aware
>>>>>> that it’s overwriting anything or not deleting properly.
>>>>>>
>>>>>> My question is - what is the intended way for the job managers to
>>>>>> correctly exchange metadata in HA mode and why is it not working for me?
>>>>>>
>>>>>> Thanks in advance!
>>>>>
>>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by Till Rohrmann <tr...@apache.org>.
Hi Encho,

thanks a lot for reporting this issue. The problem arises whenever the old
leader maintains the connection to ZooKeeper. If this is the case, then
ephemeral nodes which we create to protect against faulty delete operations
are not removed and consequently the new leader is not able to delete the
persisted job graph. So one thing to check is whether the old JM still has
an open connection to ZooKeeper. The next thing to check is the session
timeout of your ZooKeeper cluster. If you stop the job within the session
timeout, then it is also not guaranteed that ZooKeeper has detected that
the ephemeral nodes of the old JM must be deleted. In order to understand
this better it would be helpful if you could tell us the timing of the
different actions.

Cheers,
Till

On Tue, Aug 28, 2018 at 8:17 AM vino yang <ya...@gmail.com> wrote:

> Hi Encho,
>
> A temporary solution can be used to determine if it has been cleaned up by
> monitoring the specific JobID under Zookeeper's "/jobgraph".
> Another solution, modify the source code, rudely modify the cleanup mode
> to the synchronous form, but the flink operation Zookeeper's path needs to
> obtain the corresponding lock, so it is dangerous to do so, and it is not
> recommended.
> I think maybe this problem can be solved in the next version. It depends
> on Till.
>
> Thanks, vino.
>
> Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:
>
>> Thank you very much for the info! Will keep track of the progress.
>>
>> In the meantime is there any viable workaround? It seems like HA doesn't
>> really work due to this bug.
>>
>> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com> wrote:
>>
>>> About some implementation mechanisms.
>>> Flink uses Zookeeper to store JobGraph (Job's description information
>>> and metadata) as a basis for Job recovery.
>>> However, previous implementations may cause this information to not be
>>> properly cleaned up because it is asynchronously deleted by a background
>>> thread.
>>>
>>> Thanks, vino.
>>>
>>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>>
>>>> Hi Encho,
>>>>
>>>> This is a problem already known to the Flink community, you can track
>>>> its progress through FLINK-10011[1], and currently Till is fixing this
>>>> issue.
>>>>
>>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>>
>>>> Thanks, vino.
>>>>
>>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一 下午10:13写道:
>>>>
>>>>> I am running Flink 1.5.3 with two job managers and two task managers
>>>>> in Kubernetes along with HDFS and Zookeeper in high-availability mode.
>>>>>
>>>>> My problem occurs after the following actions:
>>>>> - Upload a .jar file to jobmanager-1
>>>>> - Run a streaming job from the jar on jobmanager-1
>>>>> - Wait for 1 or 2 checkpoints to succeed
>>>>> - Kill pod of jobmanager-1
>>>>> After a short delay, jobmanager-2 takes leadership and correctly
>>>>> restores the job and continues it
>>>>> - Stop job from jobmanager-2
>>>>>
>>>>> At this point all seems well, but the problem is that jobmanager-2
>>>>> does not clean up anything that was left from jobmanager-1. This means that
>>>>> both in HDFS and in Zookeeper remain job graphs, which later on obstruct
>>>>> any work of both managers as after any reset they unsuccessfully try to
>>>>> restore a non-existent job and fail over and over again.
>>>>>
>>>>> I am quite certain that jobmanager-2 does not know about any of
>>>>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>>> duplicate job folders:
>>>>>
>>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>> Error:KeeperErrorCode = NodeExists for
>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>>
>>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>> Error:KeeperErrorCode = NodeExists for
>>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>>
>>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in Zookeeper
>>>>> when the job is stopped, but fails since there are leftover files in it
>>>>> from jobmanager-1:
>>>>>
>>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>> Error:KeeperErrorCode = Directory not empty for
>>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>>
>>>>> I’ve noticed that when restoring the job, it seems like jobmanager-2
>>>>> does not get anything more than jobID, while it perhaps needs some
>>>>> metadata? Here is the log that seems suspicious to me:
>>>>>
>>>>> 2018-08-27 13:09:18,113 INFO
>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>>
>>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to be aware
>>>>> that it’s overwriting anything or not deleting properly.
>>>>>
>>>>> My question is - what is the intended way for the job managers to
>>>>> correctly exchange metadata in HA mode and why is it not working for me?
>>>>>
>>>>> Thanks in advance!
>>>>
>>>>

Re: JobGraphs not cleaned up in HA mode

Posted by vino yang <ya...@gmail.com>.
Hi Encho,

A temporary solution can be used to determine if it has been cleaned up by
monitoring the specific JobID under Zookeeper's "/jobgraph".
Another solution, modify the source code, rudely modify the cleanup mode to
the synchronous form, but the flink operation Zookeeper's path needs to
obtain the corresponding lock, so it is dangerous to do so, and it is not
recommended.
I think maybe this problem can be solved in the next version. It depends on
Till.

Thanks, vino.

Encho Mishinev <en...@gmail.com> 于2018年8月28日周二 下午1:17写道:

> Thank you very much for the info! Will keep track of the progress.
>
> In the meantime is there any viable workaround? It seems like HA doesn't
> really work due to this bug.
>
> On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com> wrote:
>
>> About some implementation mechanisms.
>> Flink uses Zookeeper to store JobGraph (Job's description information and
>> metadata) as a basis for Job recovery.
>> However, previous implementations may cause this information to not be
>> properly cleaned up because it is asynchronously deleted by a background
>> thread.
>>
>> Thanks, vino.
>>
>> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>>
>>> Hi Encho,
>>>
>>> This is a problem already known to the Flink community, you can track
>>> its progress through FLINK-10011[1], and currently Till is fixing this
>>> issue.
>>>
>>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>>
>>> Thanks, vino.
>>>
>>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一 下午10:13写道:
>>>
>>>> I am running Flink 1.5.3 with two job managers and two task managers in
>>>> Kubernetes along with HDFS and Zookeeper in high-availability mode.
>>>>
>>>> My problem occurs after the following actions:
>>>> - Upload a .jar file to jobmanager-1
>>>> - Run a streaming job from the jar on jobmanager-1
>>>> - Wait for 1 or 2 checkpoints to succeed
>>>> - Kill pod of jobmanager-1
>>>> After a short delay, jobmanager-2 takes leadership and correctly
>>>> restores the job and continues it
>>>> - Stop job from jobmanager-2
>>>>
>>>> At this point all seems well, but the problem is that jobmanager-2 does
>>>> not clean up anything that was left from jobmanager-1. This means that both
>>>> in HDFS and in Zookeeper remain job graphs, which later on obstruct any
>>>> work of both managers as after any reset they unsuccessfully try to restore
>>>> a non-existent job and fail over and over again.
>>>>
>>>> I am quite certain that jobmanager-2 does not know about any of
>>>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>>> duplicate job folders:
>>>>
>>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>> cxid:0x46 zxid:0x1ab txntype:-1 reqpath:n/a Error
>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>> Error:KeeperErrorCode = NodeExists for
>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>>
>>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:create
>>>> cxid:0x5c zxid:0x1ac txntype:-1 reqpath:n/a Error
>>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>> Error:KeeperErrorCode = NodeExists for
>>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>>
>>>> Also jobmanager-2 attempts to delete the jobgraphs folder in Zookeeper
>>>> when the job is stopped, but fails since there are leftover files in it
>>>> from jobmanager-1:
>>>>
>>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>>> cport:2181)::PrepRequestProcessor@648] - Got user-level
>>>> KeeperException when processing sessionid:0x1657aa15e480033 type:delete
>>>> cxid:0xa8 zxid:0x1bd txntype:-1 reqpath:n/a Error
>>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>> Error:KeeperErrorCode = Directory not empty for
>>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>>
>>>> I’ve noticed that when restoring the job, it seems like jobmanager-2
>>>> does not get anything more than jobID, while it perhaps needs some
>>>> metadata? Here is the log that seems suspicious to me:
>>>>
>>>> 2018-08-27 13:09:18,113 INFO
>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>>
>>>> All other logs seem fine in jobmanager-2, it doesn’t seem to be aware
>>>> that it’s overwriting anything or not deleting properly.
>>>>
>>>> My question is - what is the intended way for the job managers to
>>>> correctly exchange metadata in HA mode and why is it not working for me?
>>>>
>>>> Thanks in advance!
>>>
>>>

Re: JobGraphs not cleaned up in HA mode

Posted by Encho Mishinev <en...@gmail.com>.
Thank you very much for the info! Will keep track of the progress.

In the meantime is there any viable workaround? It seems like HA doesn't
really work due to this bug.

On Tue, Aug 28, 2018 at 4:52 AM vino yang <ya...@gmail.com> wrote:

> About some implementation mechanisms.
> Flink uses Zookeeper to store JobGraph (Job's description information and
> metadata) as a basis for Job recovery.
> However, previous implementations may cause this information to not be
> properly cleaned up because it is asynchronously deleted by a background
> thread.
>
> Thanks, vino.
>
> vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:
>
>> Hi Encho,
>>
>> This is a problem already known to the Flink community, you can track its
>> progress through FLINK-10011[1], and currently Till is fixing this issue.
>>
>> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>>
>> Thanks, vino.
>>
>> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一 下午10:13写道:
>>
>>> I am running Flink 1.5.3 with two job managers and two task managers in
>>> Kubernetes along with HDFS and Zookeeper in high-availability mode.
>>>
>>> My problem occurs after the following actions:
>>> - Upload a .jar file to jobmanager-1
>>> - Run a streaming job from the jar on jobmanager-1
>>> - Wait for 1 or 2 checkpoints to succeed
>>> - Kill pod of jobmanager-1
>>> After a short delay, jobmanager-2 takes leadership and correctly
>>> restores the job and continues it
>>> - Stop job from jobmanager-2
>>>
>>> At this point all seems well, but the problem is that jobmanager-2 does
>>> not clean up anything that was left from jobmanager-1. This means that both
>>> in HDFS and in Zookeeper remain job graphs, which later on obstruct any
>>> work of both managers as after any reset they unsuccessfully try to restore
>>> a non-existent job and fail over and over again.
>>>
>>> I am quite certain that jobmanager-2 does not know about any of
>>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>>> duplicate job folders:
>>>
>>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>>> cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException
>>> when processing sessionid:0x1657aa15e480033 type:create cxid:0x46
>>> zxid:0x1ab txntype:-1 reqpath:n/a Error
>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>> Error:KeeperErrorCode = NodeExists for
>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>>
>>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>>> cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException
>>> when processing sessionid:0x1657aa15e480033 type:create cxid:0x5c
>>> zxid:0x1ac txntype:-1 reqpath:n/a Error
>>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>> Error:KeeperErrorCode = NodeExists for
>>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>>
>>> Also jobmanager-2 attempts to delete the jobgraphs folder in Zookeeper
>>> when the job is stopped, but fails since there are leftover files in it
>>> from jobmanager-1:
>>>
>>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>>> cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException
>>> when processing sessionid:0x1657aa15e480033 type:delete cxid:0xa8
>>> zxid:0x1bd txntype:-1 reqpath:n/a Error
>>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>> Error:KeeperErrorCode = Directory not empty for
>>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>>
>>> I’ve noticed that when restoring the job, it seems like jobmanager-2
>>> does not get anything more than jobID, while it perhaps needs some
>>> metadata? Here is the log that seems suspicious to me:
>>>
>>> 2018-08-27 13:09:18,113 INFO
>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>>
>>> All other logs seem fine in jobmanager-2, it doesn’t seem to be aware
>>> that it’s overwriting anything or not deleting properly.
>>>
>>> My question is - what is the intended way for the job managers to
>>> correctly exchange metadata in HA mode and why is it not working for me?
>>>
>>> Thanks in advance!
>>
>>

Re: JobGraphs not cleaned up in HA mode

Posted by vino yang <ya...@gmail.com>.
About some implementation mechanisms.
Flink uses Zookeeper to store JobGraph (Job's description information and
metadata) as a basis for Job recovery.
However, previous implementations may cause this information to not be
properly cleaned up because it is asynchronously deleted by a background
thread.

Thanks, vino.

vino yang <ya...@gmail.com> 于2018年8月28日周二 上午9:49写道:

> Hi Encho,
>
> This is a problem already known to the Flink community, you can track its
> progress through FLINK-10011[1], and currently Till is fixing this issue.
>
> [1]: https://issues.apache.org/jira/browse/FLINK-10011
>
> Thanks, vino.
>
> Encho Mishinev <en...@gmail.com> 于2018年8月27日周一 下午10:13写道:
>
>> I am running Flink 1.5.3 with two job managers and two task managers in
>> Kubernetes along with HDFS and Zookeeper in high-availability mode.
>>
>> My problem occurs after the following actions:
>> - Upload a .jar file to jobmanager-1
>> - Run a streaming job from the jar on jobmanager-1
>> - Wait for 1 or 2 checkpoints to succeed
>> - Kill pod of jobmanager-1
>> After a short delay, jobmanager-2 takes leadership and correctly restores
>> the job and continues it
>> - Stop job from jobmanager-2
>>
>> At this point all seems well, but the problem is that jobmanager-2 does
>> not clean up anything that was left from jobmanager-1. This means that both
>> in HDFS and in Zookeeper remain job graphs, which later on obstruct any
>> work of both managers as after any reset they unsuccessfully try to restore
>> a non-existent job and fail over and over again.
>>
>> I am quite certain that jobmanager-2 does not know about any of
>> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
>> duplicate job folders:
>>
>> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
>> cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException
>> when processing sessionid:0x1657aa15e480033 type:create cxid:0x46
>> zxid:0x1ab txntype:-1 reqpath:n/a Error
>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>> Error:KeeperErrorCode = NodeExists for
>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>>
>> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
>> cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException
>> when processing sessionid:0x1657aa15e480033 type:create cxid:0x5c
>> zxid:0x1ac txntype:-1 reqpath:n/a Error
>> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>> Error:KeeperErrorCode = NodeExists for
>> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>>
>> Also jobmanager-2 attempts to delete the jobgraphs folder in Zookeeper
>> when the job is stopped, but fails since there are leftover files in it
>> from jobmanager-1:
>>
>> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
>> cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException
>> when processing sessionid:0x1657aa15e480033 type:delete cxid:0xa8
>> zxid:0x1bd txntype:-1 reqpath:n/a Error
>> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>> Error:KeeperErrorCode = Directory not empty for
>> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>>
>> I’ve noticed that when restoring the job, it seems like jobmanager-2 does
>> not get anything more than jobID, while it perhaps needs some metadata?
>> Here is the log that seems suspicious to me:
>>
>> 2018-08-27 13:09:18,113 INFO
>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>>
>> All other logs seem fine in jobmanager-2, it doesn’t seem to be aware
>> that it’s overwriting anything or not deleting properly.
>>
>> My question is - what is the intended way for the job managers to
>> correctly exchange metadata in HA mode and why is it not working for me?
>>
>> Thanks in advance!
>
>

Re: JobGraphs not cleaned up in HA mode

Posted by vino yang <ya...@gmail.com>.
Hi Encho,

This is a problem already known to the Flink community, you can track its
progress through FLINK-10011[1], and currently Till is fixing this issue.

[1]: https://issues.apache.org/jira/browse/FLINK-10011

Thanks, vino.

Encho Mishinev <en...@gmail.com> 于2018年8月27日周一 下午10:13写道:

> I am running Flink 1.5.3 with two job managers and two task managers in
> Kubernetes along with HDFS and Zookeeper in high-availability mode.
>
> My problem occurs after the following actions:
> - Upload a .jar file to jobmanager-1
> - Run a streaming job from the jar on jobmanager-1
> - Wait for 1 or 2 checkpoints to succeed
> - Kill pod of jobmanager-1
> After a short delay, jobmanager-2 takes leadership and correctly restores
> the job and continues it
> - Stop job from jobmanager-2
>
> At this point all seems well, but the problem is that jobmanager-2 does
> not clean up anything that was left from jobmanager-1. This means that both
> in HDFS and in Zookeeper remain job graphs, which later on obstruct any
> work of both managers as after any reset they unsuccessfully try to restore
> a non-existent job and fail over and over again.
>
> I am quite certain that jobmanager-2 does not know about any of
> jobmanager-1’s files since the Zookeeper logs reveal that it tries to
> duplicate job folders:
>
> 2018-08-27 13:11:00,038 [myid:] - INFO  [ProcessThread(sid:0
> cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException
> when processing sessionid:0x1657aa15e480033 type:create cxid:0x46
> zxid:0x1ab txntype:-1 reqpath:n/a Error
> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
> Error:KeeperErrorCode = NodeExists for
> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15/bbb259fd-7826-4950-bc7c-c2be23346c77
>
> 2018-08-27 13:11:02,296 [myid:] - INFO  [ProcessThread(sid:0
> cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException
> when processing sessionid:0x1657aa15e480033 type:create cxid:0x5c
> zxid:0x1ac txntype:-1 reqpath:n/a Error
> Path:/flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
> Error:KeeperErrorCode = NodeExists for
> /flink/default/checkpoint-counter/83bfa359ca59ce1d4635e18e16651e15
>
> Also jobmanager-2 attempts to delete the jobgraphs folder in Zookeeper
> when the job is stopped, but fails since there are leftover files in it
> from jobmanager-1:
>
> 2018-08-27 13:12:13,406 [myid:] - INFO  [ProcessThread(sid:0
> cport:2181)::PrepRequestProcessor@648] - Got user-level KeeperException
> when processing sessionid:0x1657aa15e480033 type:delete cxid:0xa8
> zxid:0x1bd txntype:-1 reqpath:n/a Error
> Path:/flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
> Error:KeeperErrorCode = Directory not empty for
> /flink/default/jobgraphs/83bfa359ca59ce1d4635e18e16651e15
>
> I’ve noticed that when restoring the job, it seems like jobmanager-2 does
> not get anything more than jobID, while it perhaps needs some metadata?
> Here is the log that seems suspicious to me:
>
> 2018-08-27 13:09:18,113 INFO
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
> Recovered SubmittedJobGraph(83bfa359ca59ce1d4635e18e16651e15, null).
>
> All other logs seem fine in jobmanager-2, it doesn’t seem to be aware that
> it’s overwriting anything or not deleting properly.
>
> My question is - what is the intended way for the job managers to
> correctly exchange metadata in HA mode and why is it not working for me?
>
> Thanks in advance!