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

Matthias Pohl edited comment on FLINK-34227 at 3/12/24 1:09 PM:
----------------------------------------------------------------

The [findings of my initial 
analysis|https://issues.apache.org/jira/browse/FLINK-34227?focusedCommentId=17810745&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17810745]
 are not correct. The missing log message does exist. It's just that the 
"{{Close ResourceManager connection [...]}}" log message appears twice (once 
triggered from the JobMaster's IO thread and once from the Dispatcher's main 
thread). The latter one seems to retrigger the reconnection.

{code}
[...]
02:51:28,193 [flink-pekko.actor.default-dispatcher-10] INFO  
org.apache.flink.runtime.taskexecutor.DefaultJobLeaderService [] - Remove job 
e7cb13faaae707768a1a4db28427af80 from job leader monitoring.
02:51:28,193 [flink-pekko.actor.default-dispatcher-10] INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - Close 
JobManager connection for job e7cb13faaae707768a1a4db28427af80.
02:51:28,193 [flink-pekko.actor.default-dispatcher-8] INFO  
org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotStatusSyncer [] 
- Freeing slot 98a0c702ce550d2fd7dd3710ec7b76e0.
02:51:28,194 [flink-pekko.actor.default-dispatcher-8] INFO  
org.apache.flink.runtime.jobmaster.JobMaster                 [] - Disconnect 
TaskExecutor d71ee9b8-f278-48ee-bb1c-f05fd568947f because: TaskExecutor 
pekko://flink/user/rpc/taskmanager_0 has no more allocated slots for job 
e7cb13faaae707768a1a4db28427af80.
02:51:28,194 [jobmanager-io-thread-3] INFO  
org.apache.flink.runtime.jobmaster.JobMaster                 [] - Close 
ResourceManager connection 3c08958c5ef3906fae847097373b047a: Stopping JobMaster 
for job 'Flink Streaming Job' (e7cb13faaae707768a1a4db28427af80).
02:51:28,194 [flink-pekko.actor.default-dispatcher-5] INFO  
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager [] - 
Disconnect job manager 
a38b8b4ba6c4894c7cfca5f1c0fe4f68@pekko://flink/user/rpc/jobmanager_70 for job 
e7cb13faaae707768a1a4db28427af80 from the resource manager.
02:51:28,194 [flink-pekko.actor.default-dispatcher-5] INFO  
org.apache.flink.runtime.jobmaster.JobMaster                 [] - Close 
ResourceManager connection 3c08958c5ef3906fae847097373b047a: Stopping JobMaster 
for job 'Flink Streaming Job' (e7cb13faaae707768a1a4db28427af80).
02:51:28,194 [flink-pekko.actor.default-dispatcher-5] INFO  
org.apache.flink.runtime.jobmaster.JobMaster                 [] - Connecting to 
ResourceManager 
pekko://flink/user/rpc/resourcemanager_2(86dfd2ebd79836698df3e4a5de474282)
02:51:28,194 [flink-pekko.actor.default-dispatcher-5] INFO  
org.apache.flink.runtime.jobmaster.JobMaster                 [] - Resolved 
ResourceManager address, beginning registration
02:51:28,194 [flink-pekko.actor.default-dispatcher-5] INFO  
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager [] - 
Registering job manager 
a38b8b4ba6c4894c7cfca5f1c0fe4f68@pekko://flink/user/rpc/jobmanager_70 for job 
e7cb13faaae707768a1a4db28427af80.
02:51:28,195 [flink-pekko.actor.default-dispatcher-5] INFO  
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager [] - 
Registered job manager 
a38b8b4ba6c4894c7cfca5f1c0fe4f68@pekko://flink/user/rpc/jobmanager_70 for job 
e7cb13faaae707768a1a4db28427af80.
02:51:28,195 [flink-pekko.actor.default-dispatcher-5] INFO  
org.apache.flink.runtime.jobmaster.JobMaster                 [] - JobManager 
successfully registered at ResourceManager, leader id: 
86dfd2ebd79836698df3e4a5de474282.
[...]
{code}


was (Author: mapohl):
The [findings of my initial 
analysis|https://issues.apache.org/jira/browse/FLINK-34227?focusedCommentId=17810745&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17810745]
 are not correct. The missing log message does exist. It's just that the 
"{{Close ResourceManager connection [...]}}" log message appears twice (once 
triggered from the JobMaster's IO thread and once from the Dispatcher's main 
thread). The latter one seems to retrigger the reconnection.

> 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