[ https://issues.apache.org/jira/browse/FLINK-34227?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17810453#comment-17810453 ]
Matthias Pohl commented on FLINK-34227: --------------------------------------- I attached the logs of the {{WindowDistinctAggregateITCase}} test runs (FLINK-34227.log) and an extract only focusing on the test run that created job 7e7d69daebb438b8d03b7392c9c55115 and the subsequent test run up to the point where the missing resources for two jobs are logged for the first time ((FLINK-34227.7e7d69daebb438b8d03b7392c9c55115.log). There's something odd going on when shutting down the JobMaster/deregistering the JobMaster from the ResourceManager: {code} [...] 06:01:26,703 [flink-pekko.actor.default-dispatcher-10] INFO org.apache.flink.runtime.jobmaster.JobMaster [] - Disconnect TaskExecutor 7d5dbc6f-47ad-4043-ba88-b8cadefe0ca5 because: TaskExecutor pekko://flink/user/rpc/taskmanager_0 has no more allocated slots for job 7e7d69daebb438b8d03b7392c9c55115. 06:01:26,703 [flink-pekko.actor.default-dispatcher-10] INFO org.apache.flink.runtime.jobmaster.JobMaster [] - Close ResourceManager connection bb1ce04485fc5291a97cd5d488709ff9: Stopping JobMaster for job 'Flink Streaming Job' (7e7d69daebb438b8d03b7392c9c55115). 06:01:26,703 [flink-pekko.actor.default-dispatcher-10] INFO org.apache.flink.runtime.jobmaster.JobMaster [] - Connecting to ResourceManager pekko://flink/user/rpc/resourcemanager_2(891f46bbb398d49ad91e1dde0bee410c) [...] {code} The disconnect is not finalized but a connection is re-established. Expected: {code} 06:01:24,830 [flink-pekko.actor.default-dispatcher-9] INFO org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Close JobManager connection for job 7f7374259a708b71336253292acd9c18. 06:01:24,830 [flink-pekko.actor.default-dispatcher-9] INFO org.apache.flink.runtime.resourcemanager.StandaloneResourceManager [] - Disconnect job manager 808cbb235481cba25abb51ada4d4447a@pekko://flink/user/rpc/jobmanager_15 for job 7f7374259a708b71336253292acd9c18 from the resourc e manager. 06:01:24,830 [flink-pekko.actor.default-dispatcher-9] INFO org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotStatusSyncer [] - Freeing slot 372f94cc8f615aac8257dab9e2367557. 06:01:24,830 [flink-pekko.actor.default-dispatcher-9] INFO org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotStatusSyncer [] - Freeing slot 88011a68b4e67c12f4410ac784ce17d1. 06:01:24,830 [flink-pekko.actor.default-dispatcher-9] INFO org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotStatusSyncer [] - Freeing slot b7b967686260841ceee74680e9d93459. 06:01:24,831 [flink-pekko.actor.default-dispatcher-9] INFO org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotStatusSyncer [] - Freeing slot 0cb83b2f13abfe914fc483af87c5009f. {code} > Job is not properly cleaned up in FineGrainedSlotManager > -------------------------------------------------------- > > Key: FLINK-34227 > URL: https://issues.apache.org/jira/browse/FLINK-34227 > Project: Flink > Issue Type: Bug > Components: Runtime / Coordination > Affects Versions: 1.18.1 > Reporter: Matthias Pohl > Priority: Major > 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)