[ 
https://issues.apache.org/jira/browse/FLINK-22516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17343317#comment-17343317
 ] 

Ricky Burnett commented on FLINK-22516:
---------------------------------------

Yep, that's pretty much where I ended up in my investigation.  There is nothing 
interesting in the Zookeeper logs.  Just some global session commits and a 
snapshot.   Sadly we don't have debug logs.   I do have some additional 
information on that `Unhandled exception`.  It's logged in the 
`org.apache.flink.runtime.rest.handler.job.JobIdsHandler`. Here is the stack 
trace.  I am not able to suss out what is going on here.

 
{code:java}
 extendedStackTrace: akka.pattern.AskTimeoutException: Ask timed out on 
[Actor[akka://flink/user/dispatcher#-1916238732]] after [60000 ms]. Message of 
type [org.apache.flink.runtime.rpc.messages.LocalFencedMessage]. A typical 
reason for `AskTimeoutException` is that the recipient actor didn't send a 
reply.
        at akka.pattern.PromiseActorRef$$anonfun$2.apply(AskSupport.scala:635) 
~[flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at akka.pattern.PromiseActorRef$$anonfun$2.apply(AskSupport.scala:635) 
~[flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at 
akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:648) 
[flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:205) 
[flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at 
scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
 [flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at 
scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109) 
[flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at 
scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599) 
[flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at 
akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:328)
 [flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at 
akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:279)
 [flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at 
akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:283)
 [flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at 
akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:235)
 [flink-extensions_2.11-1.9.3-splunk-2.4-2.4-all.jar:?]
        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_282]

     localizedMessage: Ask timed out on 
[Actor[akka://flink/user/dispatcher#-1916238732]] after [60000 ms]. Message of 
type [org.apache.flink.runtime.rpc.messages.LocalFencedMessage]. A typical 
reason for `AskTimeoutException` is that the recipient actor didn't send a 
reply.
     message: Ask timed out on 
[Actor[akka://flink/user/dispatcher#-1916238732]] after [60000 ms]. Message of 
type [org.apache.flink.runtime.rpc.messages.LocalFencedMessage]. A typical 
reason for `AskTimeoutException` is that the recipient actor didn't send a 
reply.
     name: akka.pattern.AskTimeoutException
{code}

> ResourceManager cannot establish leadership
> -------------------------------------------
>
>                 Key: FLINK-22516
>                 URL: https://issues.apache.org/jira/browse/FLINK-22516
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.9.3
>         Environment: 1.9.3 Flink version, on kubernetes.
>            Reporter: Ricky Burnett
>            Priority: Major
>         Attachments: jm1.log, jm2.log, jobmanager_leadership.log
>
>
> We are running Flink clusters with 2 Jobmanagers in HA mode.  After a 
> Zookeeper restart the two JMs begin leadership election end up in state where 
> they are both trying to start their ResourceManager and until one of them 
> writes to `leader/<jobid>/resource_manager_lock` and the other Jobmanager's 
> JobMaster proceeds to execute `notifyOfNewResourceManagerLeader` which 
> restarts the ResourceManager.  This in turn writes to 
> `leader/<jobid>/resource_manager_lock` which triggers the first JobMaster to 
> restart it's ResourceManager.  We can see this in the logs from the 
> "ResourceManager leader changed to new address" log, that goes back and forth 
> between the two JMs and the two IP addresses.  This cycle appears to continue 
> indefinitely with outside interruption.  
> I've attached combined logs from two JMs in our environment that got into 
> this state.  The logs start with the loss of connection and end with a couple 
> of cycles of back and forth.   The two relevant hosts are 
> "flink-jm-828d4aa2-d4d4-457b-995d-feb56d08c1fb-784cdb9c57-tsxb7" and 
> "flink-jm-828d4aa2-d4d4-457b-995d-feb56d08c1fb-784cdb9c57-mpf9x".
> *-tsxb7 appears to be the last host that was granted leadership. 
> {code:java}
> {"thread":"Curator-Framework-0-EventThread","level":"INFO","loggerName":"org.apache.flink.runtime.jobmaster.JobManagerRunner","message":"JobManager
>  runner for job tenant: ssademo, pipeline: 
> 828d4aa2-d4d4-457b-995d-feb56d08c1fb, name: integration-test-detection 
> (33e12948df69077ab3b33316eacbb5e4) was granted leadership with session id 
> 97992805-9c60-40ba-8260-aaf036694cde at 
> akka.tcp://[email protected]:6123/user/jobmanager_3.","endOfBatch":false,"loggerFqcn":"org.apache.logging.slf4j.Log4jLogger","instant":{"epochSecond":1617129712,"nanoOfSecond":447000000},"contextMap":{},"threadId":152,"threadPriority":5,"source":{"class":"org.apache.flink.runtime.jobmaster.JobManagerRunner","method":"startJobMaster","file":"JobManagerRunner.java","line":313},"service":"streams","time":"2021-03-30T18:41:52.447UTC","hostname":"flink-jm-828d4aa2-d4d4-457b-995d-feb56d08c1fb-784cdb9c57-tsxb7"}
> {code}
> But  *-mpf9x continues to try to wrestle control back.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to