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

Matthias Pohl edited comment on FLINK-34227 at 3/13/24 12:28 PM:
-----------------------------------------------------------------

One other theory on that issue:
I agree that we should fix the {{runAfterwards}} issue you raised. But I 
continued investigating it because I found it strange that it only appears in 
the {{AdaptiveScheduler}} test profile even though we're using the same logic 
in 
[SchedulerBase#close|https://github.com/apache/flink/blob/d4e0084649c019c536ee1e44bab15c8eca01bf13/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java#L674]
 as mentioned in the previous comment.

But now on another theory: The problem seems to be that the {{JobMaster}} 
disconnecting from the {{ResourceManager}} happens twice. The second disconnect 
triggers a reconnect and re-registration of the {{JobMaster}} in the 
{{ResourceManager}}. This can theoretically happen because the first disconnect 
will trigger a {{JobMaster#disconnectResourceManager}} in 
[ResourceManager#closeJobManagerConnection in line 
1150|https://github.com/apache/flink/blob/d6a4eb966fbc47277e07b79e7c64939a62eb1d54/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java#L1150].
 If this RPC call makes it to the JobMaster before its RPCEndpoint is shutdown, 
it will get processed once more leading to the reconnect because the 
{{JobMaster#resourceManagerAddress}} is still set (which is the condition for 
{{JobMaster#isConnectingToResourceManager}} which is called in 
[JobMaster#disconnectResourceManager|https://github.com/apache/flink/blob/7d0111dfab640f2f590dd710d76de927c86cf83e/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java#L847]).
 With the {{resourceManagerAddress}} being set and pointing to the correct 
{{ResourceManager}}, reconnection would be triggered.


was (Author: mapohl):
One other theory on that issue:
I agree that we should fix the {{runAfterwards}} issue you raised. But I 
continued investigating it because I found it strange that it only appears in 
the {{AdaptiveScheduler}} test profile even though we're using the same logic 
in 
[SchedulerBase#close|https://github.com/apache/flink/blob/d4e0084649c019c536ee1e44bab15c8eca01bf13/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java#L674]
 as mentioned in the previous comment.

But now on another theory: The problem seems to be that the {{JobMaster}} 
disconnecting from the {{ResourceManager}} happens twice. The second disconnect 
triggers a reconnect and re-registration of the {{JobMaster}} in the 
{{ResourceManager}}. This can theoretically happen because the first disconnect 
will trigger a {{JobMaster#disconnectResourceManager}} in 
[ResourceManager#closeJobManagerConnection in line 
1150|https://github.com/apache/flink/blob/d6a4eb966fbc47277e07b79e7c64939a62eb1d54/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java#L1150].
 If this RPC call makes it to the JobMaster before its RPCEndpoint is shutdown, 
it will get processed once more leading to the reconnect because the 
{{JobMaster#resourceManagerAddress}} is still set (which is the condition for 
{{JobMaster#isConnectingToResourceManager}} which is called in 
[JobMaster#disconnectResourceManager|https://github.com/apache/flink/blob/7d0111dfab640f2f590dd710d76de927c86cf83e/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java#L847]).

> Job doesn't disconnect from ResourceManager
> -------------------------------------------
>
>                 Key: FLINK-34227
>                 URL: https://issues.apache.org/jira/browse/FLINK-34227
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.19.0, 1.18.1
>            Reporter: Matthias Pohl
>            Assignee: Matthias Pohl
>            Priority: Critical
>              Labels: github-actions, test-stability
>         Attachments: FLINK-34227.7e7d69daebb438b8d03b7392c9c55115.log, 
> FLINK-34227.log
>
>
> https://github.com/XComp/flink/actions/runs/7634987973/job/20800205972#step:10:14557
> {code}
> [...]
> "main" #1 prio=5 os_prio=0 tid=0x00007fcccc4b7000 nid=0x24ec0 waiting on 
> condition [0x00007fccce1eb000]
>    java.lang.Thread.State: WAITING (parking)
>       at sun.misc.Unsafe.park(Native Method)
>       - parking to wait for  <0x00000000bdd52618> (a 
> java.util.concurrent.CompletableFuture$Signaller)
>       at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>       at 
> java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1707)
>       at 
> java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
>       at 
> java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1742)
>       at 
> java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
>       at 
> org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:2131)
>       at 
> org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:2099)
>       at 
> org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:2077)
>       at 
> org.apache.flink.streaming.api.scala.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.scala:876)
>       at 
> org.apache.flink.table.planner.runtime.stream.sql.WindowDistinctAggregateITCase.testHopWindow_Cube(WindowDistinctAggregateITCase.scala:550)
> [...]
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to