Sorry I mixed up the log, it belongs to previous failure.

Could you trying to reproduce the problem with DEBUG level log?

>From the log we knew that JM & RM had been elected as leader but the
listener didn't work. However, we didn't know it is because the leader
didn't publish the leader info or the listener didn't get notified.

Best,
tison.


tison <wander4...@gmail.com> 于2020年3月18日周三 上午10:40写道:

> Hi Abhinav,
>
> The problem is
>
> Curator: Background operation retry gave up
>
> So it is the ZK ensemble too unstable to get recovery in time so that
> Curator stopped retrying and threw a fatal error.
>
> Best,
> tison.
>
>
> Xintong Song <tonysong...@gmail.com> 于2020年3月18日周三 上午10:22写道:
>
>> I'm not familiar with ZK either.
>>
>> I've copied Yang Wang, who might be able to provide some suggestions.
>>
>> Alternatively, you can try to post your question to the Apache ZooKeeper
>> community, see if they have any clue.
>>
>> Thank you~
>>
>> Xintong Song
>>
>>
>>
>> On Wed, Mar 18, 2020 at 8:12 AM Bajaj, Abhinav <abhinav.ba...@here.com>
>> wrote:
>>
>>> Hi Xintong,
>>>
>>>
>>>
>>> I did check the Zk logs and didn’t notice anything interesting.
>>>
>>> I have limited expertise in zookeeper.
>>>
>>> Can you share an example of what I should be looking for in Zk?
>>>
>>>
>>>
>>> I was able to reproduce this issue again with Flink 1.7 by killing the
>>> zookeeper leader that disrupted the quorum.
>>>
>>> The sequence of logs in this case look quite similar to one we have been
>>> discussing.
>>>
>>>
>>>
>>> If the code hasn’t changed in this area till 1.10 then maybe the latest
>>> version also has the potential issue.
>>>
>>>
>>>
>>> Its not straightforward to bump up the Flink version in the
>>> infrastructure available to me.
>>>
>>> But I will think if there is a way around it.
>>>
>>>
>>>
>>> ~ Abhinav Bajaj
>>>
>>>
>>>
>>> *From: *Xintong Song <tonysong...@gmail.com>
>>> *Date: *Monday, March 16, 2020 at 8:00 PM
>>> *To: *"Bajaj, Abhinav" <abhinav.ba...@here.com>
>>> *Cc: *"user@flink.apache.org" <user@flink.apache.org>
>>> *Subject: *Re: JobMaster does not register with ResourceManager in high
>>> availability setup
>>>
>>>
>>>
>>> Hi Abhinav,
>>>
>>>
>>>
>>> I think you are right. The log confirms that JobMaster has not tried to
>>> connect ResourceManager. Most likely the JobMaster requested for RM address
>>> but has never received it.
>>>
>>>
>>>
>>> I would suggest you to check the ZK logs, see if the request form JM for
>>> RM address has been received and properly responded.
>>>
>>>
>>>
>>> If you can easily reproduce this problem, and you are able to build
>>> Flink from source, you can also try to insert more logs in Flink to further
>>> confirm whether the RM address is received. I don't think that's necessary
>>> though, since those codes have not been changed since Flink 1.7 till the
>>> latest 1.10, and I'm not aware of any reported issue that the JM may not
>>> try to connect RM once the address is received.
>>>
>>>
>>>
>>> Thank you~
>>>
>>> Xintong Song
>>>
>>>
>>>
>>>
>>>
>>> On Tue, Mar 17, 2020 at 7:45 AM Bajaj, Abhinav <abhinav.ba...@here.com>
>>> wrote:
>>>
>>> Hi Xintong,
>>>
>>>
>>>
>>> Apologies for delayed response. I was away for a week.
>>>
>>> I am attaching more jobmanager logs.
>>>
>>>
>>>
>>> To your point on the taskmanagers, the job is deployed with 20
>>> parallelism but it has 22 TMs to have 2 of them as spare to assist in quick
>>> failover.
>>>
>>> I did check the logs and all 22 of task executors from those TMs get
>>> registered by the time - 2020-02-27 06:35:47.050.
>>>
>>>
>>>
>>> You would notice that even after this time, the job fails with the error
>>> “org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException:
>>> Could not allocate all requires slots within timeout of 300000 ms. Slots
>>> required: 201, slots allocated: 0” at 2020-02-27 06:40:36.778.
>>>
>>>
>>>
>>> Thanks a ton for you help.
>>>
>>>
>>>
>>> ~ Abhinav Bajaj
>>>
>>>
>>>
>>> *From: *Xintong Song <tonysong...@gmail.com>
>>> *Date: *Thursday, March 5, 2020 at 6:30 PM
>>> *To: *"Bajaj, Abhinav" <abhinav.ba...@here.com>
>>> *Cc: *"user@flink.apache.org" <user@flink.apache.org>
>>> *Subject: *Re: JobMaster does not register with ResourceManager in high
>>> availability setup
>>>
>>>
>>>
>>> Hi Abhinav,
>>>
>>>
>>>
>>> Thanks for the log. However, the attached log seems to be incomplete.
>>> The NoResourceAvailableException cannot be found in this log.
>>>
>>>
>>>
>>> Regarding connecting to ResourceManager, the log suggests that:
>>>
>>>    - ZK was back to life and connected at 06:29:56.
>>>    2020-02-27 06:29:56.539 [main-EventThread] level=INFO
>>>     o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State
>>>    change: CONNECTED
>>>    - RM registered to ZK and was granted leadership at 06:30:01.
>>>    2020-02-27 06:30:01.677 [flink-akka.actor.default-dispatcher-5]
>>>    level=INFO  o.a.flink.runtime.resourcemanager.StandaloneResourceManager  
>>> -
>>>    ResourceManager akka.tcp://flink@JOBMANAGER:6126/user/resourcemanager
>>>    was granted leadership with fencing token 
>>> a2c453481ea4e0c7722cab1e4dd741db
>>>    - JM requests RM leader address from ZK at 06:30:06.
>>>    2020-02-27 06:30:06.272 [flink-akka.actor.default-dispatcher-17]
>>>    level=INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  -
>>>    Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>>>    - The RM leader address will be notified asynchronously, and only
>>>    after that JM will try to connect to RM (printing the "Connecting to
>>>    ResourceManager" log). The attached log ends in 100ms after JM requesting
>>>    RM leader address, which is too short to tell whether the RM is connected
>>>    properly.
>>>
>>> Another finding is about the TM registration. According to the log:
>>>
>>>    - The parallelism of your job is 20, which means it needs 20 slots
>>>    to be executed.
>>>    - There are only 5 TMs registered. (Searching for "Registering
>>>    TaskManager with ResourceID")
>>>    - Assuming you have the same configurations for JM and TMs (this
>>>    might not always be true), you have one slot per TM.
>>>    599 2020-02-27 06:28:56.495 [main] level=INFO
>>>     org.apache.flink.configuration.GlobalConfiguration  - Loading
>>>    configuration property: taskmanager.numberOfTaskSlots, 1
>>>    - That suggests that it is possible that not all the TaskExecutors
>>>    are recovered/reconnected, leading to the NoResourceAvailableException. 
>>> We
>>>    would need the rest part of the log (from where the current one ends to
>>>    the NoResourceAvailableException) to tell what happened during the
>>>    scheduling. Also, could you confirm how many TMs do you use?
>>>
>>>
>>>
>>>
>>> Thank you~
>>>
>>> Xintong Song
>>>
>>>
>>>
>>>
>>>
>>> On Fri, Mar 6, 2020 at 5:55 AM Bajaj, Abhinav <abhinav.ba...@here.com>
>>> wrote:
>>>
>>> Hi Xintong,
>>>
>>>
>>>
>>> Highly appreciate your assistance here.
>>>
>>> I am attaching the jobmanager log for reference.
>>>
>>>
>>>
>>> Let me share my quick responses on what you mentioned.
>>>
>>>
>>>
>>>
>>>
>>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Cannot serve
>>> slot request, no ResourceManager connected.
>>>
>>> *XS: Sometimes you see this log because the ResourceManager is not yet
>>> connect when the slot request arrives the SlotPool. If the ResourceManager
>>> is connected later, the SlotPool will still send the pending slot requests,
>>> in that case you should find logs for SlotPool requesting slots from
>>> ResourceManager.*
>>>
>>>
>>>
>>> *AB*: Yes, I have noticed that behavior in scenarios where
>>> resourcemanager and jobmanager are connected successfully. The requests
>>> fail initially and they are served later when they are connected.
>>>
>>> I don’t think that happened in this case. But you have access to the
>>> jobmanager logs to check my understanding.
>>>
>>>
>>>
>>> org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException:
>>> Could not allocate all requires slots within timeout of 300000 ms……
>>>
>>> *XS: This error message simply means that the slot requests are not
>>> satisfied in 5min. Various reasons might cause this problem.*
>>>
>>>    - *The ResourceManager is not connected at all.*
>>>
>>>
>>>    - *AB*: I think Resoucemanager is not connected to Jobmaster or vice
>>>       versa. My basis is *absence* of below logs –
>>>
>>>
>>>    - org.apache.flink.runtime.jobmaster.JobMaster  - Connecting to
>>>          ResourceManager....
>>>          - o.a.flink.runtime.resourcemanager.StandaloneResourceManager
>>>          - Registering job manager....
>>>
>>>
>>>    - *The ResourceManager is connected, but some TaskExecutors are not
>>>    registered due to the ZK problem. *
>>>
>>>
>>>    - *AB*: I think the Task Executors were able to register or were in
>>>       the process of registering with ResourceManager.
>>>
>>>
>>>    - *ZK recovery takes too much time, so that despite all JM, RM, TMs
>>>    are able to connect to the ZK there might not be enough time to satisfy 
>>> the
>>>    slot request before the timeout.*
>>>
>>>
>>>    - *AB*: To help check that may be you can use this log time
>>>
>>>
>>>    - 2020-02-27 06:29:53,732 [myid:1] - INFO
>>>          [QuorumPeer[myid=1]/0.0.0.0:2181:Follower@64] - FOLLOWING -
>>>          LEADER ELECTION TOOK - 25069
>>>          - 2020-02-27 06:29:53,766 [myid:1] - INFO
>>>          [QuorumPeer[myid=1]/0.0.0.0:2181:Learner@332] - Getting a diff
>>>          from the leader 0x200002bf6
>>>
>>> Thanks a lot for looking into this.
>>>
>>> ~ Abhinav Bajaj
>>>
>>>
>>>
>>>
>>>
>>> *From: *Xintong Song <tonysong...@gmail.com>
>>> *Date: *Wednesday, March 4, 2020 at 7:17 PM
>>> *To: *"Bajaj, Abhinav" <abhinav.ba...@here.com>
>>> *Cc: *"user@flink.apache.org" <user@flink.apache.org>
>>> *Subject: *Re: JobMaster does not register with ResourceManager in high
>>> availability setup
>>>
>>>
>>>
>>> Hi Abhinav,
>>>
>>>
>>>
>>> Do you mind sharing the complete 'jobmanager.log'?
>>>
>>>
>>>
>>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Cannot serve
>>> slot request, no ResourceManager connected.
>>>
>>> Sometimes you see this log because the ResourceManager is not yet
>>> connect when the slot request arrives the SlotPool. If the ResourceManager
>>> is connected later, the SlotPool will still send the pending slot requests,
>>> in that case you should find logs for SlotPool requesting slots from
>>> ResourceManager.
>>>
>>>
>>>
>>> org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException:
>>> Could not allocate all requires slots within timeout of 300000 ms……
>>>
>>> This error message simply means that the slot requests are not satisfied
>>> in 5min. Various reasons might cause this problem.
>>>
>>>    - The ResourceManager is not connected at all.
>>>    - The ResourceManager is connected, but some TaskExecutors are not
>>>    registered due to the ZK problem.
>>>    - ZK recovery takes too much time, so that despite all JM, RM, TMs
>>>    are able to connect to the ZK there might not be enough time to satisfy 
>>> the
>>>    slot request before the timeout.
>>>
>>> It would need the complete 'jobmanager.log' (at least those from the job
>>> restart to the NoResourceAvailableException) to find out which is the case.
>>>
>>>
>>> Thank you~
>>>
>>> Xintong Song
>>>
>>>
>>>
>>>
>>>
>>> On Thu, Mar 5, 2020 at 7:30 AM Bajaj, Abhinav <abhinav.ba...@here.com>
>>> wrote:
>>>
>>> While I setup to reproduce the issue with debug logs, I would like to
>>> share more information I noticed in INFO logs.
>>>
>>>
>>>
>>> Below is the sequence of events/exceptions I notice during the time
>>> zookeeper was disrupted.
>>>
>>> I apologize in advance as they are a bit verbose.
>>>
>>>
>>>
>>>    - Zookeeper seems to be down and leader election is disrupted –
>>>
>>>
>>>
>>> ·         2020-02-27 06:28:23.572 [Curator-ConnectionStateManager-0]
>>> level=WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  -
>>> Connection to ZooKeeper suspended. The contender 
>>> akka.tcp://flink@FOO_BAR:6126/user/resourcemanager
>>> no longer participates in the leader election.
>>>
>>> ·         2020-02-27 06:28:23.573 [Curator-ConnectionStateManager-0]
>>> level=INFO  org.apache.flink.runtime.jobmaster.JobManagerRunner  -
>>> JobManager for job FOO_BAR (5a910928a71b469a091be168b0e74722) was revoked
>>> leadership at akka.tcp://flink@ FOO_BAR:6126/user/jobmanager_1.
>>>
>>> ·         2020-02-27 06:28:23.573
>>> [flink-akka.actor.default-dispatcher-9897] level=INFO
>>> o.a.flink.runtime.resourcemanager.StandaloneResourceManager  -
>>> ResourceManager akka.tcp://flink@ FOO_BAR:6126/user/resourcemanager was
>>> revoked leadership. Clearing fencing token.
>>>
>>> ·         2020-02-27 06:28:23.574
>>> [flink-akka.actor.default-dispatcher-9897] level=INFO
>>> o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Suspending the
>>> SlotManager.
>>>
>>> ·         2020-02-27 06:28:53.577 [Curator-Framework-0] level=ERROR
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint  - Fatal error
>>> occurred in the cluster entrypoint.
>>>
>>> org.apache.flink.runtime.dispatcher.DispatcherException: Received an
>>> error from the LeaderElectionService.
>>>
>>>         at
>>> org.apache.flink.runtime.dispatcher.Dispatcher.handleError(Dispatcher.java:941)
>>>
>>>         at
>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService.unhandledError(ZooKeeperLeaderElectionService.java:416)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl$6.apply(CuratorFrameworkImpl.java:576)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl$6.apply(CuratorFrameworkImpl.java:572)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:93)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer.forEach(ListenerContainer.java:85)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.logError(CuratorFrameworkImpl.java:571)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.checkBackgroundRetry(CuratorFrameworkImpl.java:740)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.performBackgroundOperation(CuratorFrameworkImpl.java:857)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.backgroundOperationsLoop(CuratorFrameworkImpl.java:809)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.access$300(CuratorFrameworkImpl.java:64)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl$4.call(CuratorFrameworkImpl.java:267)
>>>
>>>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>
>>>         at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>>>
>>>         at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>>
>>>         at
>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>>>
>>>         at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>>>
>>>         at java.lang.Thread.run(Thread.java:748)
>>>
>>> Caused by: org.apache.flink.util.FlinkException: Unhandled error in
>>> ZooKeeperLeaderElectionService: Background operation retry gave up
>>>
>>>         ... 18 common frames omitted
>>>
>>> Caused by:
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.KeeperException$ConnectionLossException:
>>> KeeperErrorCode = ConnectionLoss
>>>
>>>         at
>>> org.apache.flink.shaded.zookeeper.org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.checkBackgroundRetry(CuratorFrameworkImpl.java:728)
>>>
>>>         ... 10 common frames omitted
>>>
>>>
>>>
>>>    - ClusterEntrypoint restarts and tries to connect to Zookeeper. It
>>>    seems its fails for some time but able to connect later -
>>>
>>>
>>>
>>> ·         2020-02-27 06:28:56.467 [main] level=INFO
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint  -  Starting
>>> StandaloneSessionClusterEntrypoint (Version: 1.7.1, Rev:89eafb4,
>>> Date:14.12.2018 @ 15:48:34 GMT)
>>>
>>> ·         2020-02-27 06:29:16.477 [main] level=ERROR
>>> o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Connection
>>> timed out for connection string (ZOO_BAR_0:2181, ZOO_BAR_1:2181,
>>> ZOO_BAR_2:2181) and timeout (15000) / elapsed (15969)
>>>
>>> org.apache.flink.shaded.curator.org.apache.curator.CuratorConnectionLossException:
>>> KeeperErrorCode = ConnectionLoss
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:225)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:94)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:117)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.NamespaceImpl$1.call(NamespaceImpl.java:90)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.RetryLoop.callWithRetry(RetryLoop.java:109)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.NamespaceImpl.fixForNamespace(NamespaceImpl.java:83)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.CuratorFrameworkImpl.fixForNamespace(CuratorFrameworkImpl.java:594)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.ExistsBuilderImpl.forPath(ExistsBuilderImpl.java:158)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.imps.ExistsBuilderImpl.forPath(ExistsBuilderImpl.java:32)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.cache.NodeCache.reset(NodeCache.java:242)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.cache.NodeCache.start(NodeCache.java:175)
>>>
>>>         at
>>> org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.cache.NodeCache.start(NodeCache.java:154)
>>>
>>>         at
>>> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService.start(ZooKeeperLeaderElectionService.java:134)
>>>
>>>         at
>>> org.apache.flink.runtime.webmonitor.WebMonitorEndpoint.startInternal(WebMonitorEndpoint.java:712)
>>>
>>>         at
>>> org.apache.flink.runtime.rest.RestServerEndpoint.start(RestServerEndpoint.java:218)
>>>
>>>         at
>>> org.apache.flink.runtime.entrypoint.component.AbstractDispatcherResourceManagerComponentFactory.create(AbstractDispatcherResourceManagerComponentFactory.java:145)
>>>
>>>         at
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:215)
>>>
>>>         at
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$0(ClusterEntrypoint.java:163)
>>>
>>>         at
>>> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
>>>
>>>         at
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:162)
>>>
>>>         at
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runClusterEntrypoint(ClusterEntrypoint.java:517)
>>>
>>>         at
>>> org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint.main(StandaloneSessionClusterEntrypoint.java:65)
>>>
>>> ·         2020-02-27 06:30:01.643 [main] level=INFO
>>> o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Starting
>>> ZooKeeperLeaderElectionService
>>> ZooKeeperLeaderElectionService{leaderPath='/leader/resource_manager_lock'}.
>>>
>>> ·         2020-02-27 06:30:01.655 [main] level=INFO
>>> o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Starting
>>> ZooKeeperLeaderElectionService
>>> ZooKeeperLeaderElectionService{leaderPath='/leader/dispatcher_lock'}.
>>>
>>> ·         2020-02-27 06:30:01.677
>>> [flink-akka.actor.default-dispatcher-16] level=INFO
>>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Dispatcher
>>> akka.tcp://flink@FOO_BAR:6126/user/dispatcher was granted leadership
>>> with fencing token 113d78b5-6c33-401b-9f47-2f7a1d6dfefc
>>>
>>> ·         2020-02-27 06:30:01.677
>>> [flink-akka.actor.default-dispatcher-5] level=INFO
>>> o.a.flink.runtime.resourcemanager.StandaloneResourceManager  -
>>> ResourceManager akka.tcp://flink@FOO_BAR:6126/user/resourcemanager was
>>> granted leadership with fencing token a2c453481ea4e0c7722cab1e4dd741db
>>>
>>> ·         2020-02-27 06:30:06.251 [main-EventThread] level=INFO
>>> org.apache.flink.runtime.jobmaster.JobManagerRunner  - JobManager runner
>>> for job HHW_WEATHER_PROCESSOR (5a910928a71b469a091be168b0e74722) was
>>> granted leadership with session id 32637c78-cfd4-4e44-8c10-c551bac40742 at
>>> akka.tcp://flink@FOO_BAR:6126/user/jobmanager_0.
>>>
>>>
>>>
>>> My zookeeper knowledge is a bit limited but I do notice that the below
>>> log from zookeeper instance came back up and joined the quorum as follower
>>> before the above highlighted logs on jobmanager side.
>>>
>>> ·         2020-02-27 06:29:53,732 [myid:1] - INFO
>>> [QuorumPeer[myid=1]/0.0.0.0:2181:Follower@64] - FOLLOWING - LEADER
>>> ELECTION TOOK - 25069
>>>
>>> ·         2020-02-27 06:29:53,766 [myid:1] - INFO
>>> [QuorumPeer[myid=1]/0.0.0.0:2181:Learner@332] - Getting a diff from the
>>> leader 0x200002bf6
>>>
>>>
>>>
>>>
>>>
>>> I will setup to reproduce this issue and get debug logs as well.
>>>
>>>
>>>
>>> But in meantime, does the above hightlighted logs confirm that zookeeper
>>> become available around that time?
>>>
>>> I don’t see any logs from JobMaster complaining for not being able to
>>> connect to zookeeper after that.
>>>
>>>
>>>
>>> ~ Abhinav Bajaj
>>>
>>>
>>>
>>> *From: *"Bajaj, Abhinav" <abhinav.ba...@here.com>
>>> *Date: *Wednesday, March 4, 2020 at 12:01 PM
>>> *To: *Xintong Song <tonysong...@gmail.com>
>>> *Cc: *"user@flink.apache.org" <user@flink.apache.org>
>>> *Subject: *Re: JobMaster does not register with ResourceManager in high
>>> availability setup
>>>
>>>
>>>
>>> Thanks Xintong for pointing that out.
>>>
>>>
>>>
>>> I will dig deeper and get back with my findings.
>>>
>>>
>>>
>>> ~ Abhinav Bajaj
>>>
>>>
>>>
>>> *From: *Xintong Song <tonysong...@gmail.com>
>>> *Date: *Tuesday, March 3, 2020 at 7:36 PM
>>> *To: *"Bajaj, Abhinav" <abhinav.ba...@here.com>
>>> *Cc: *"user@flink.apache.org" <user@flink.apache.org>
>>> *Subject: *Re: JobMaster does not register with ResourceManager in high
>>> availability setup
>>>
>>>
>>>
>>> Hi Abhinav,
>>>
>>>
>>> The JobMaster log "Connecting to ResourceManager ..." is printed after
>>> JobMaster retrieve ResourceManager address from ZooKeeper. In your case, I
>>> assume there's some ZK problem that JM cannot resolve RM address.
>>>
>>>
>>>
>>> Have you confirmed whether the ZK pods are recovered after the second
>>> disruption? And does the address changed?
>>>
>>>
>>>
>>> You can also try to enable debug logs for the following components, to
>>> see if there's any useful information.
>>>
>>> org.apache.flink.runtime.jobmaster
>>>
>>> org.apache.flink.runtime.resourcemanager
>>>
>>> org.apache.flink.runtime.highavailability
>>>
>>> org.apache.flink.runtime.leaderretrieval
>>>
>>> org.apache.zookeeper
>>>
>>>
>>>
>>> Thank you~
>>>
>>> Xintong Song
>>>
>>>
>>>
>>>
>>>
>>> On Wed, Mar 4, 2020 at 5:42 AM Bajaj, Abhinav <abhinav.ba...@here.com>
>>> wrote:
>>>
>>> Hi,
>>>
>>>
>>>
>>> We recently came across an issue where JobMaster does not register with
>>> ResourceManager in Fink high availability setup.
>>>
>>> Let me share the details below.
>>>
>>>
>>>
>>> *Setup*
>>>
>>>    - Flink 1.7.1
>>>    - K8s
>>>    - High availability mode with a *single* Jobmanager and 3 zookeeper
>>>    nodes in quorum.
>>>
>>>
>>>
>>> *Scenario*
>>>
>>>    - Zookeeper pods are disrupted by K8s that leads to resetting of
>>>    leadership of JobMaster & ResourceManager and restart of the Flink job.
>>>
>>>
>>>
>>> *Observations*
>>>
>>>    - After the first disruption of Zookeeper, JobMaster and
>>>    ResourceManager were reset & were able to register with each other. 
>>> Sharing
>>>    few logs that confirm that. Flink job restarted successfully.
>>>
>>> org.apache.flink.runtime.jobmaster.JobMaster  - Connecting to
>>> ResourceManager....
>>>
>>> o.a.flink.runtime.resourcemanager.StandaloneResourceManager  -
>>> Registering job manager....
>>>
>>> o.a.flink.runtime.resourcemanager.StandaloneResourceManager  -
>>> Registered job manager....
>>>
>>> org.apache.flink.runtime.jobmaster.JobMaster  - JobManager successfully
>>> registered at ResourceManager...
>>>
>>>    -  After another disruption later on the same Flink
>>>    cluster, JobMaster & ResourceManager were not connected and below logs 
>>> can
>>>    be noticed and eventually scheduler times out.
>>>
>>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Cannot serve
>>> slot request, no ResourceManager connected.
>>>
>>>        ………
>>>
>>>         
>>> org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: 
>>> Could not allocate all requires slots within timeout of 300000 ms……
>>>
>>>
>>>    - I can confirm from the logs that both JobMaster & ResourceManager
>>>    were running. JobMaster was trying to recover the job and ResourceManager
>>>    registered the taskmanagers.
>>>    - The odd thing is that the log for JobMaster trying to connect to
>>>    ResourceManager is missing. So I assume JobMaster didn’t try to connect 
>>> to
>>>    ResourceManager.
>>>
>>>
>>>
>>> I can share more logs if required.
>>>
>>>
>>>
>>> Has anyone noticed similar behavior or is this a known issue with Flink
>>> 1.7.1?
>>>
>>> Any recommendations or suggestions on fix or workaround?
>>>
>>>
>>>
>>> Appreciate your time and help here.
>>>
>>>
>>>
>>> ~ Abhinav Bajaj
>>>
>>>
>>>
>>>
>>>
>>>

Reply via email to