[
https://issues.apache.org/jira/browse/FLINK-36451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17895936#comment-17895936
]
Matthias Pohl edited comment on FLINK-36451 at 11/27/24 2:34 PM:
-----------------------------------------------------------------
The problem is that we have two code paths running concurrently (in the
contenders main thread and in the main thread of the
{{DefaultLeaderElectionService}}) that acquire the same locks in reversed order:
* Certain operations in the contender (the DispatcherLeaderProcess in
FLINK-36451 and similarly the JobMasterServiceLeadershipRunner) need to be
called only as leader. These operations are executed from within the contender
and acquire the contender’s lock first before acquiring the
LeaderElectionService lock (to check for leadership)).
* In the meantime, any leadership change event will trigger the same locks in
reverse order: First, locking the DefaultLeaderElectionService lock and then
informing the registered contenders. Notifying the contenders will require
acquiring their locks as part of the process.
With unstable leadership happening quite often, the chance of running into a
deadlock increases. This is an issue that does not only affect the k8s but also
the ZooKeeper leader election.
The solution should be to run any leader election-related operations solely on
the {{DefaultLeaderElectionService}} main thread to ensure sequential
execution. This allows us to remove the lock within the
{{DefaultLeaderElectionService}}. We have to ensure, though, that the
operations that are performed on the {{DefaultLeaderElectionService}} main
thread are light-weight (similar to the requirements of Flink's {{RpcEndpoint}}
main thread).
was (Author: mapohl):
The problem is that we have two code paths running concurrently (in the
contenders main thread and in the main thread of the
{{DefaultLeaderElectionService}}) that acquire the same locks in reversed order:
* Certain operations in the contender (the DispatcherLeaderProcess in
FLINK-36451 and the JobMasterServiceLeadershipRunner) need to be called only as
leader. These operations are executed from within the contender and acquire the
contender’s lock first before acquiring the LeaderElectionService lock (to
check for leadership)).
* In the meantime, any leadership change event will trigger the same locks in
reverse order: First, locking the DefaultLeaderElectionService lock and then
informing the registered contenders. Notifying the contenders will require
acquiring their locks as part of the process.
With unstable leadership happening quite often, the chance of running into a
deadlock increases. This is an issue that does not only affect the k8s but also
the ZooKeeper leader election.
The solution should be to run any leader election-related operations solely on
the {{DefaultLeaderElectionService}} main thread to ensure sequential
execution. This allows us to remove the lock within the
{{DefaultLeaderElectionService}}. We have to ensure, though, that the
operations that are performed on the {{DefaultLeaderElectionService}} main
thread are light-weight (similar to the requirements of Flink's {{RpcEndpoint}}
main thread).
> Kubernetes Application JobManager Potential Deadlock and TaskManager Pod
> Residuals
> ----------------------------------------------------------------------------------
>
> Key: FLINK-36451
> URL: https://issues.apache.org/jira/browse/FLINK-36451
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Coordination
> Affects Versions: 1.19.1
> Environment: * Flink version: 1.19.1
> * - Deployment mode: Flink Kubernetes Application Mode
> * - JVM version: OpenJDK 17
>
> Reporter: xiechenling
> Assignee: Matthias Pohl
> Priority: Major
> Labels: pull-request-available
> Attachments: 1.png, 2.png, jobmanager.log, jstack.txt
>
>
> In Kubernetes Application Mode, when there is significant etcd latency or
> instability, the Flink JobManager may enter a deadlock situation.
> Additionally, TaskManager pods are not cleaned up properly, resulting in
> stale resources that prevent the Flink job from recovering correctly. This
> issue occurs during frequent service restarts or network instability.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)