[
https://issues.apache.org/jira/browse/FLINK-30878?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17683327#comment-17683327
]
Matthias Pohl commented on FLINK-30878:
---------------------------------------
{code:java}
Java stack information for the threads listed above:
===================================================
"config-map-watch-handler-thread-2851":
at
org.apache.flink.kubernetes.kubeclient.resources.KubernetesSharedInformer$WatchCallback.lambda$run$0(KubernetesSharedInformer.java:243)
- waiting to lock <0x00000000d14d1928> (a java.lang.Object)
at
org.apache.flink.kubernetes.kubeclient.resources.KubernetesSharedInformer$WatchCallback$$Lambda$879/261416860.run(Unknown
Source)
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:750)
"config-map-watch-handler-thread-4":
at
org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService.notifyAllKnownLeaderInformation(DefaultMultipleComponentLeaderElectionService.java:264)
- waiting to lock <0x00000000d14d6bd0> (a java.lang.Object)
at
org.apache.flink.kubernetes.highavailability.KubernetesMultipleComponentLeaderElectionDriver$ConfigMapCallbackHandlerImpl.onModified(KubernetesMultipleComponentLeaderElectionDriver.java:238)
at
org.apache.flink.kubernetes.kubeclient.resources.KubernetesSharedInformer$EventHandler.lambda$null$3(KubernetesSharedInformer.java:208)
at
org.apache.flink.kubernetes.kubeclient.resources.KubernetesSharedInformer$EventHandler$$Lambda$945/45359867.accept(Unknown
Source)
at
org.apache.flink.kubernetes.kubeclient.resources.KubernetesSharedInformer$WatchCallback.lambda$run$0(KubernetesSharedInformer.java:246)
- locked <0x00000000d14d1928> (a java.lang.Object)
at
org.apache.flink.kubernetes.kubeclient.resources.KubernetesSharedInformer$WatchCallback$$Lambda$879/261416860.run(Unknown
Source)
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:750)
"leadershipOperationExecutor-thread-1":
at
org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService.onLeaderInformationChange(DefaultLeaderElectionService.java:254)
- waiting to lock <0x00000000d14d6bf8> (a java.lang.Object)
at
org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService.lambda$sendLeaderInformationChange$3(DefaultMultipleComponentLeaderElectionService.java:254)
- locked <0x00000000d14d6bd0> (a java.lang.Object)
at
org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService$$Lambda$948/665822281.run(Unknown
Source)
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:750)
"mini-cluster-io-thread-2":
at
org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService.hasLeadership(DefaultMultipleComponentLeaderElectionService.java:180)
- waiting to lock <0x00000000d14d6bd0> (a java.lang.Object)
at
org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriverAdapter.hasLeadership(MultipleComponentLeaderElectionDriverAdapter.java:51)
at
org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService.hasLeadership(DefaultLeaderElectionService.java:155)
- locked <0x00000000d14d6bf8> (a java.lang.Object)
at
org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService.confirmLeadership(DefaultLeaderElectionService.java:120)
- locked <0x00000000d14d6bf8> (a java.lang.Object)
at
org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunner.lambda$forwardConfirmLeaderSessionFuture$2(DefaultDispatcherRunner.java:175)
at
org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunner$$Lambda$886/1947395283.accept(Unknown
Source)
at
java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
at
java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
{code}
The deadlock occurs due to the lock being acquired in FLINK-30474 before
triggering the {{onLeaderInformationChange}} in
[DefaultMultipleComponentLeaderElectionService.java
:254|https://github.com/apache/flink/pull/21537/commits/dbce30603bc9436032a98ae1b9d33f99e0790099#diff-8e1b9ed8178f117c8e5f3be67ce6f5b341862d06ce27f14d52901b503f6a3a0bR254]
> KubernetesHighAvailabilityRecoverFromSavepointITCase fails due to a deadlock
> ----------------------------------------------------------------------------
>
> Key: FLINK-30878
> URL: https://issues.apache.org/jira/browse/FLINK-30878
> Project: Flink
> Issue Type: Bug
> Components: Deployment / Kubernetes, Runtime / Coordination
> Affects Versions: 1.17.0, 1.15.3, 1.16.1
> Reporter: Matthias Pohl
> Assignee: Matthias Pohl
> Priority: Blocker
> Labels: test-stability
>
> We're seeing a test failure in
> {{KubernetesHighAvailabilityRecoverFromSavepointITCase}} due to a deadlock:
> {code:java}
> 2023-02-01T18:53:35.5540322Z "ForkJoinPool-1-worker-1" #14 daemon prio=5
> os_prio=0 tid=0x00007f68ecb18000 nid=0x43dd1 waiting on condition
> [0x00007f68c1711000]
> 2023-02-01T18:53:35.5540900Z java.lang.Thread.State: TIMED_WAITING
> (parking)
> 2023-02-01T18:53:35.5541272Z at sun.misc.Unsafe.park(Native Method)
> 2023-02-01T18:53:35.5541932Z - parking to wait for <0x00000000d14d7b60> (a
> java.util.concurrent.CompletableFuture$Signaller)
> 2023-02-01T18:53:35.5542496Z at
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
> 2023-02-01T18:53:35.5543088Z at
> java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1709)
> 2023-02-01T18:53:35.5543672Z at
> java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3313)
> 2023-02-01T18:53:35.5544240Z at
> java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1788)
> 2023-02-01T18:53:35.5544801Z at
> java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
> 2023-02-01T18:53:35.5545632Z at
> org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityRecoverFromSavepointITCase.testRecoverFromSavepoint(KubernetesHighAvailabilityRecoverFromSavepointITCase.java:113)
> 2023-02-01T18:53:35.5546409Z at
> sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) {code}
> [https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=45565&view=logs&j=bea52777-eaf8-5663-8482-18fbc3630e81&t=b2642e3a-5b86-574d-4c8a-f7e2842bfb14&l=61916]
> The build failure happens on 1.16. I'm adding 1.17 and 1.15 as fixVersions as
> well because it might be due to some recent changes which were introduced
> with FLINK-30462 and/or FLINK-30474
--
This message was sent by Atlassian Jira
(v8.20.10#820010)