You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Manjusha Vuyyuru <vm...@gmail.com> on 2019/05/07 11:46:06 UTC

flink 1.7 HA production setup going down completely

Hello,

I have a flink setup with two job managers coordinated by zookeeper.

I see the below exception and both jobmanagers are going down:

2019-05-07 08:29:13,346 INFO
org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
Released locks of job graph f8eb1b482d8ec8c1d3e94c4d0f79df77 from ZooKeeper.
2019-05-07 08:29:13,346 ERROR
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -* Fatal
error occurred in the cluster entrypoint.*
java.lang.RuntimeException: 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.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
        at
org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:74)
        at
java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
        at
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
        at
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
        at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
        at
akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
        at
scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
        at
scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
        at
scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
        at
scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
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


Can someone please help me understand in detail on what is causing this
exception. I can see zookeeper not able to retrieve job graph. What could
be the reason for this?

This is second time that my setup is going down with this excepton, first
time i cleared jobgraph folder in zookeeper and restarted, now again faced
with same issue.

Since this is production setup this way of outage is not at all expected
:(. Can someone help me how to give a permanent fix to this issue?


Thanks,
Manju

Re: flink 1.7 HA production setup going down completely

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

I guess this exception

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)
at
org.apache.hadoop.hdfs.DFSInputStream.chooseDataNode(DFSInputStream.java:1036)
at
org.apache.hadoop.hdfs.DFSInputStream.chooseDataNode(DFSInputStream.java:1015)
at
org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:647)
at
org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:926)
at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:982)
at java.io.DataInputStream.read(DataInputStream.java:149)
at
org.apache.flink.runtime.fs.hdfs.HadoopDataInputStream.read(HadoopDataInputStream.java:94)
at
java.io.ObjectInputStream$PeekInputStream.read(ObjectInputStream.java:2620)
at
java.io.ObjectInputStream$PeekInputStream.readFully(ObjectInputStream.java:2636)
at
java.io.ObjectInputStream$BlockDataInputStream.readShort(ObjectInputStream.java:3113)
at java.io.ObjectInputStream.readStreamHeader(ObjectInputStream.java:853)
at java.io.ObjectInputStream.<init>(ObjectInputStream.java:349)
at
org.apache.flink.util.InstantiationUtil$ClassLoaderObjectInputStream.<init>(InstantiationUtil.java:68)
at
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:520)
at
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:503)
at
org.apache.flink.runtime.state.RetrievableStreamStateHandle.retrieveState(RetrievableStreamStateHandle.java:58)
at
org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore.recoverJobGraph(ZooKeeperSubmittedJobGraphStore.java:202)

and the following log statements

2019-05-07 08:28:54,136 WARN  org.apache.hadoop.hdfs.DFSClient
                - No live nodes contain block
BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 after
checking nodes = [], ignoredNodes = null
2019-05-07 08:28:54,137 INFO  org.apache.hadoop.hdfs.DFSClient
                - No node available for
BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494
file=/flink/checkpoints/submittedJobGraph480ddf9572ed
2019-05-07 08:28:54,137 INFO  org.apache.hadoop.hdfs.DFSClient
                - Could not obtain
BP-1651346363-10.20.1.81-1525354906737:blk_1083182315_9441494 from any
node:  No live nodes contain current block Block locations: Dead nodes: .
Will get new block locations from namenode and retry...
2019-05-07 08:28:54,137 WARN  org.apache.hadoop.hdfs.DFSClient
                - DFS chooseDataNode: got # 1 IOException, will wait for
1498.8531884268646 msec.

pretty much explain what's happening. Flink cannot read all the blocks
belonging to the submitted job graph file and fails due to this. This looks
like a HDFS problem to me.

Cheers,
Till

On Wed, May 8, 2019 at 4:59 PM Manjusha Vuyyuru <vm...@gmail.com>
wrote:

> Hi Till,
> Thanks for the response.
> please see the attached log file.
>
> *HA config is : *
> high-availability: zookeeper
> high-availability.storageDir: hdfs://flink-hdfs:9000/flink/checkpoints
> From the logs i can see block missing exceptions from hdfs, but i can see
> that the jobgraph is still present in hdfs.
>
>
>
> On Wed, May 8, 2019 at 7:56 PM Till Rohrmann <tr...@apache.org> wrote:
>
>> Hi Manju,
>>
>> could you share the full logs or at least the full stack trace of the
>> exception with us?
>>
>> I suspect that after a failover Flink tries to restore the JobGraph from
>> persistent storage (the directory which you have configured via
>> `high-availability.storageDir`) but is not able to do so. One reason could
>> be that the JobGraph file has been removed by a third party, for example. I
>> think the cause of the FlinkException could shed light on it. Could you
>> verify that the JobGraph file is still accessible?
>>
>> Cheers,
>> Till
>>
>> On Wed, May 8, 2019 at 11:22 AM Manjusha Vuyyuru <vm...@gmail.com>
>> wrote:
>>
>>> Any update on this from community side?
>>>
>>> On Tue, May 7, 2019 at 6:43 PM Manjusha Vuyyuru <vm...@gmail.com>
>>> wrote:
>>>
>>>> im using 1.7.2.
>>>>
>>>>
>>>> On Tue, May 7, 2019 at 5:50 PM miki haiat <mi...@gmail.com> wrote:
>>>>
>>>>> Which flink version are you using?
>>>>> I had similar  issues with 1.5.x
>>>>>
>>>>> On Tue, May 7, 2019 at 2:49 PM Manjusha Vuyyuru <
>>>>> vmanjusha.cse@gmail.com> wrote:
>>>>>
>>>>>> Hello,
>>>>>>
>>>>>> I have a flink setup with two job managers coordinated by zookeeper.
>>>>>>
>>>>>> I see the below exception and both jobmanagers are going down:
>>>>>>
>>>>>> 2019-05-07 08:29:13,346 INFO
>>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>>> Released locks of job graph f8eb1b482d8ec8c1d3e94c4d0f79df77 from ZooKeeper.
>>>>>> 2019-05-07 08:29:13,346 ERROR
>>>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -*
>>>>>> Fatal error occurred in the cluster entrypoint.*
>>>>>> java.lang.RuntimeException: 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.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
>>>>>>         at
>>>>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:74)
>>>>>>         at
>>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
>>>>>>         at
>>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>>         at
>>>>>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
>>>>>>         at
>>>>>> akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
>>>>>>         at
>>>>>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
>>>>>>         at
>>>>>> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>>>>>>         at
>>>>>> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>>>>>>         at
>>>>>> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>>>>>>         at
>>>>>> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
>>>>>> 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
>>>>>>
>>>>>>
>>>>>> Can someone please help me understand in detail on what is causing
>>>>>> this exception. I can see zookeeper not able to retrieve job graph. What
>>>>>> could be the reason for this?
>>>>>>
>>>>>> This is second time that my setup is going down with this excepton,
>>>>>> first time i cleared jobgraph folder in zookeeper and restarted, now again
>>>>>> faced with same issue.
>>>>>>
>>>>>> Since this is production setup this way of outage is not at all
>>>>>> expected :(. Can someone help me how to give a permanent fix to this issue?
>>>>>>
>>>>>>
>>>>>> Thanks,
>>>>>> Manju
>>>>>>
>>>>>>

Re: flink 1.7 HA production setup going down completely

Posted by Manjusha Vuyyuru <vm...@gmail.com>.
Hi Till,
Thanks for the response.
please see the attached log file.

*HA config is : *
high-availability: zookeeper
high-availability.storageDir: hdfs://flink-hdfs:9000/flink/checkpoints
From the logs i can see block missing exceptions from hdfs, but i can see
that the jobgraph is still present in hdfs.



On Wed, May 8, 2019 at 7:56 PM Till Rohrmann <tr...@apache.org> wrote:

> Hi Manju,
>
> could you share the full logs or at least the full stack trace of the
> exception with us?
>
> I suspect that after a failover Flink tries to restore the JobGraph from
> persistent storage (the directory which you have configured via
> `high-availability.storageDir`) but is not able to do so. One reason could
> be that the JobGraph file has been removed by a third party, for example. I
> think the cause of the FlinkException could shed light on it. Could you
> verify that the JobGraph file is still accessible?
>
> Cheers,
> Till
>
> On Wed, May 8, 2019 at 11:22 AM Manjusha Vuyyuru <vm...@gmail.com>
> wrote:
>
>> Any update on this from community side?
>>
>> On Tue, May 7, 2019 at 6:43 PM Manjusha Vuyyuru <vm...@gmail.com>
>> wrote:
>>
>>> im using 1.7.2.
>>>
>>>
>>> On Tue, May 7, 2019 at 5:50 PM miki haiat <mi...@gmail.com> wrote:
>>>
>>>> Which flink version are you using?
>>>> I had similar  issues with 1.5.x
>>>>
>>>> On Tue, May 7, 2019 at 2:49 PM Manjusha Vuyyuru <
>>>> vmanjusha.cse@gmail.com> wrote:
>>>>
>>>>> Hello,
>>>>>
>>>>> I have a flink setup with two job managers coordinated by zookeeper.
>>>>>
>>>>> I see the below exception and both jobmanagers are going down:
>>>>>
>>>>> 2019-05-07 08:29:13,346 INFO
>>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>>> Released locks of job graph f8eb1b482d8ec8c1d3e94c4d0f79df77 from ZooKeeper.
>>>>> 2019-05-07 08:29:13,346 ERROR
>>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -*
>>>>> Fatal error occurred in the cluster entrypoint.*
>>>>> java.lang.RuntimeException: 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.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
>>>>>         at
>>>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:74)
>>>>>         at
>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
>>>>>         at
>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>         at
>>>>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
>>>>>         at
>>>>> akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
>>>>>         at
>>>>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
>>>>>         at
>>>>> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>>>>>         at
>>>>> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>>>>>         at
>>>>> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>>>>>         at
>>>>> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
>>>>> 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
>>>>>
>>>>>
>>>>> Can someone please help me understand in detail on what is causing
>>>>> this exception. I can see zookeeper not able to retrieve job graph. What
>>>>> could be the reason for this?
>>>>>
>>>>> This is second time that my setup is going down with this excepton,
>>>>> first time i cleared jobgraph folder in zookeeper and restarted, now again
>>>>> faced with same issue.
>>>>>
>>>>> Since this is production setup this way of outage is not at all
>>>>> expected :(. Can someone help me how to give a permanent fix to this issue?
>>>>>
>>>>>
>>>>> Thanks,
>>>>> Manju
>>>>>
>>>>>

Re: flink 1.7 HA production setup going down completely

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

could you share the full logs or at least the full stack trace of the
exception with us?

I suspect that after a failover Flink tries to restore the JobGraph from
persistent storage (the directory which you have configured via
`high-availability.storageDir`) but is not able to do so. One reason could
be that the JobGraph file has been removed by a third party, for example. I
think the cause of the FlinkException could shed light on it. Could you
verify that the JobGraph file is still accessible?

Cheers,
Till

On Wed, May 8, 2019 at 11:22 AM Manjusha Vuyyuru <vm...@gmail.com>
wrote:

> Any update on this from community side?
>
> On Tue, May 7, 2019 at 6:43 PM Manjusha Vuyyuru <vm...@gmail.com>
> wrote:
>
>> im using 1.7.2.
>>
>>
>> On Tue, May 7, 2019 at 5:50 PM miki haiat <mi...@gmail.com> wrote:
>>
>>> Which flink version are you using?
>>> I had similar  issues with 1.5.x
>>>
>>> On Tue, May 7, 2019 at 2:49 PM Manjusha Vuyyuru <vm...@gmail.com>
>>> wrote:
>>>
>>>> Hello,
>>>>
>>>> I have a flink setup with two job managers coordinated by zookeeper.
>>>>
>>>> I see the below exception and both jobmanagers are going down:
>>>>
>>>> 2019-05-07 08:29:13,346 INFO
>>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>>> Released locks of job graph f8eb1b482d8ec8c1d3e94c4d0f79df77 from ZooKeeper.
>>>> 2019-05-07 08:29:13,346 ERROR
>>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -* Fatal
>>>> error occurred in the cluster entrypoint.*
>>>> java.lang.RuntimeException: 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.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
>>>>         at
>>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:74)
>>>>         at
>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
>>>>         at
>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>         at
>>>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
>>>>         at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
>>>>         at
>>>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
>>>>         at
>>>> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>>>>         at
>>>> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>>>>         at
>>>> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>>>>         at
>>>> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
>>>> 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
>>>>
>>>>
>>>> Can someone please help me understand in detail on what is causing this
>>>> exception. I can see zookeeper not able to retrieve job graph. What could
>>>> be the reason for this?
>>>>
>>>> This is second time that my setup is going down with this excepton,
>>>> first time i cleared jobgraph folder in zookeeper and restarted, now again
>>>> faced with same issue.
>>>>
>>>> Since this is production setup this way of outage is not at all
>>>> expected :(. Can someone help me how to give a permanent fix to this issue?
>>>>
>>>>
>>>> Thanks,
>>>> Manju
>>>>
>>>>

Re: flink 1.7 HA production setup going down completely

Posted by Manjusha Vuyyuru <vm...@gmail.com>.
Any update on this from community side?

On Tue, May 7, 2019 at 6:43 PM Manjusha Vuyyuru <vm...@gmail.com>
wrote:

> im using 1.7.2.
>
>
> On Tue, May 7, 2019 at 5:50 PM miki haiat <mi...@gmail.com> wrote:
>
>> Which flink version are you using?
>> I had similar  issues with 1.5.x
>>
>> On Tue, May 7, 2019 at 2:49 PM Manjusha Vuyyuru <vm...@gmail.com>
>> wrote:
>>
>>> Hello,
>>>
>>> I have a flink setup with two job managers coordinated by zookeeper.
>>>
>>> I see the below exception and both jobmanagers are going down:
>>>
>>> 2019-05-07 08:29:13,346 INFO
>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>>> Released locks of job graph f8eb1b482d8ec8c1d3e94c4d0f79df77 from ZooKeeper.
>>> 2019-05-07 08:29:13,346 ERROR
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -* Fatal
>>> error occurred in the cluster entrypoint.*
>>> java.lang.RuntimeException: 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.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
>>>         at
>>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:74)
>>>         at
>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
>>>         at
>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>         at
>>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
>>>         at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
>>>         at
>>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
>>>         at
>>> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>>>         at
>>> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>>>         at
>>> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>>>         at
>>> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
>>> 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
>>>
>>>
>>> Can someone please help me understand in detail on what is causing this
>>> exception. I can see zookeeper not able to retrieve job graph. What could
>>> be the reason for this?
>>>
>>> This is second time that my setup is going down with this excepton,
>>> first time i cleared jobgraph folder in zookeeper and restarted, now again
>>> faced with same issue.
>>>
>>> Since this is production setup this way of outage is not at all expected
>>> :(. Can someone help me how to give a permanent fix to this issue?
>>>
>>>
>>> Thanks,
>>> Manju
>>>
>>>

Re: flink 1.7 HA production setup going down completely

Posted by Manjusha Vuyyuru <vm...@gmail.com>.
im using 1.7.2.


On Tue, May 7, 2019 at 5:50 PM miki haiat <mi...@gmail.com> wrote:

> Which flink version are you using?
> I had similar  issues with 1.5.x
>
> On Tue, May 7, 2019 at 2:49 PM Manjusha Vuyyuru <vm...@gmail.com>
> wrote:
>
>> Hello,
>>
>> I have a flink setup with two job managers coordinated by zookeeper.
>>
>> I see the below exception and both jobmanagers are going down:
>>
>> 2019-05-07 08:29:13,346 INFO
>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
>> Released locks of job graph f8eb1b482d8ec8c1d3e94c4d0f79df77 from ZooKeeper.
>> 2019-05-07 08:29:13,346 ERROR
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -* Fatal
>> error occurred in the cluster entrypoint.*
>> java.lang.RuntimeException: 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.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
>>         at
>> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:74)
>>         at
>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
>>         at
>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>         at
>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
>>         at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
>>         at
>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
>>         at
>> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>>         at
>> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>>         at
>> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>>         at
>> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
>> 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
>>
>>
>> Can someone please help me understand in detail on what is causing this
>> exception. I can see zookeeper not able to retrieve job graph. What could
>> be the reason for this?
>>
>> This is second time that my setup is going down with this excepton, first
>> time i cleared jobgraph folder in zookeeper and restarted, now again faced
>> with same issue.
>>
>> Since this is production setup this way of outage is not at all expected
>> :(. Can someone help me how to give a permanent fix to this issue?
>>
>>
>> Thanks,
>> Manju
>>
>>

Re: flink 1.7 HA production setup going down completely

Posted by miki haiat <mi...@gmail.com>.
Which flink version are you using?
I had similar  issues with 1.5.x

On Tue, May 7, 2019 at 2:49 PM Manjusha Vuyyuru <vm...@gmail.com>
wrote:

> Hello,
>
> I have a flink setup with two job managers coordinated by zookeeper.
>
> I see the below exception and both jobmanagers are going down:
>
> 2019-05-07 08:29:13,346 INFO
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
> Released locks of job graph f8eb1b482d8ec8c1d3e94c4d0f79df77 from ZooKeeper.
> 2019-05-07 08:29:13,346 ERROR
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -* Fatal
> error occurred in the cluster entrypoint.*
> java.lang.RuntimeException: 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.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
>         at
> org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:74)
>         at
> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
>         at
> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>         at
> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
>         at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
>         at
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
>         at
> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>         at
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>         at
> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>         at
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> 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
>
>
> Can someone please help me understand in detail on what is causing this
> exception. I can see zookeeper not able to retrieve job graph. What could
> be the reason for this?
>
> This is second time that my setup is going down with this excepton, first
> time i cleared jobgraph folder in zookeeper and restarted, now again faced
> with same issue.
>
> Since this is production setup this way of outage is not at all expected
> :(. Can someone help me how to give a permanent fix to this issue?
>
>
> Thanks,
> Manju
>
>