[
https://issues.apache.org/jira/browse/FLINK-34227?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17825698#comment-17825698
]
Matthias Pohl commented on FLINK-34227:
---------------------------------------
[~chesnay] supported the investigation. His findings are based around the
question why the close call was actually triggered from within an IO thread:
# The {{JobMaster#onStop}} call triggers
[JobMaster#stopJobExecution|https://github.com/apache/flink/blob/d6a4eb966fbc47277e07b79e7c64939a62eb1d54/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java#L1048]
which calls {{JobMaster#stopScheduling}} from within the JobMaster's main
thread which calls {{AdaptiveScheduler#closeAsync}} in
[JobMaster:1088|https://github.com/apache/flink/blob/d6a4eb966fbc47277e07b79e7c64939a62eb1d54/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java#L1088].
# {{AdaptiveScheduler#closeAsync}} composes the resultFuture of the
{{CheckpointsCleaner}} in
[AdaptiveScheduler:581|https://github.com/apache/flink/blob/d4e0084649c019c536ee1e44bab15c8eca01bf13/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java#L581].
The future is completed from within the {{ioExecutor}} that is used in the
{{CheckpointsCleaner}}.
# The future is forwarded to {{JobMaster#stopJobExecution}} where the
disconnect is triggered after the cleanup future is completed causing the
disconnect to be executed in an IO thread rather than the JobMaster's main
thread.
> 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)