[
https://issues.apache.org/jira/browse/FLINK-34427?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816969#comment-17816969
]
Chesnay Schepler commented on FLINK-34427:
------------------------------------------
The problem is the use of scheduled executors in the FineGrainedSlotManager. It
periodically tries to schedule actions unconditionally into the main thread,
and this periodic action is also never cancelled.
If the rpc endpoint shuts down during the periodic delay the scheduled action
can fire again before the rpc service (and thus scheduled executor) is shut
down, running into this error.
This code is plain broken as tt makes assumptions about the lifecycle of the
scheduled executor. The loop should be canceled when the FGSM is shut down, and
as a safety rail any scheduled action should validate that the FGSM is not shut
down yet before scheduling anything into the main thread.
> FineGrainedSlotManagerTest fails fatally (exit code 239)
> --------------------------------------------------------
>
> Key: FLINK-34427
> URL: https://issues.apache.org/jira/browse/FLINK-34427
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Coordination
> Affects Versions: 1.19.0, 1.20.0
> Reporter: Matthias Pohl
> Assignee: Matthias Pohl
> Priority: Critical
> Labels: test-stability
>
> https://github.com/apache/flink/actions/runs/7866453350/job/21460921911#step:10:8959
> {code}
> Error: 02:28:53 02:28:53.220 [ERROR] Process Exit Code: 239
> Error: 02:28:53 02:28:53.220 [ERROR] Crashed tests:
> Error: 02:28:53 02:28:53.220 [ERROR]
> org.apache.flink.runtime.resourcemanager.ResourceManagerTaskExecutorTest
> Error: 02:28:53 02:28:53.220 [ERROR]
> org.apache.maven.surefire.booter.SurefireBooterForkException:
> ExecutionException The forked VM terminated without properly saying goodbye.
> VM crash or System.exit called?
> Error: 02:28:53 02:28:53.220 [ERROR] Command was /bin/sh -c cd
> '/root/flink/flink-runtime' &&
> '/usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java' '-XX:+UseG1GC' '-Xms256m'
> '-XX:+IgnoreUnrecognizedVMOptions'
> '--add-opens=java.base/java.util=ALL-UNNAMED'
> '--add-opens=java.base/java.lang=ALL-UNNAMED'
> '--add-opens=java.base/java.net=ALL-UNNAMED'
> '--add-opens=java.base/java.io=ALL-UNNAMED'
> '--add-opens=java.base/java.util.concurrent=ALL-UNNAMED' '-Xmx768m' '-jar'
> '/root/flink/flink-runtime/target/surefire/surefirebooter-20240212022332296_94.jar'
> '/root/flink/flink-runtime/target/surefire'
> '2024-02-12T02-21-39_495-jvmRun3' 'surefire-20240212022332296_88tmp'
> 'surefire_26-20240212022332296_91tmp'
> Error: 02:28:53 02:28:53.220 [ERROR] Error occurred in starting fork, check
> output in log
> Error: 02:28:53 02:28:53.220 [ERROR] Process Exit Code: 239
> Error: 02:28:53 02:28:53.220 [ERROR] Crashed tests:
> Error: 02:28:53 02:28:53.221 [ERROR]
> org.apache.flink.runtime.resourcemanager.ResourceManagerTaskExecutorTest
> Error: 02:28:53 02:28:53.221 [ERROR] at
> org.apache.maven.plugin.surefire.booterclient.ForkStarter.awaitResultsDone(ForkStarter.java:456)
> [...]
> {code}
> The fatal error is triggered most likely within the
> {{FineGrainedSlotManagerTest}}:
> {code}
> 02:26:39,362 [ pool-643-thread-1] ERROR
> org.apache.flink.util.FatalExitExceptionHandler [] - FATAL:
> Thread 'pool-643-thread-1' produced an uncaught exception. Stopping the
> process...
> java.util.concurrent.CompletionException:
> java.util.concurrent.RejectedExecutionException: Task
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@4bbc0b10
> rejected from
> java.util.concurrent.ScheduledThreadPoolExecutor@7a45cd9a[Shutting down, pool
> size = 1, active threads = 1, queued tasks = 1, completed tasks = 194]
> at
> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:273)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:280)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:838)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:811)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.CompletableFuture.uniHandleStage(CompletableFuture.java:851)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.CompletableFuture.handleAsync(CompletableFuture.java:2178)
> ~[?:1.8.0_392]
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotStatusSyncer.allocateSlot(DefaultSlotStatusSyncer.java:138)
> ~[classes/:?]
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.FineGrainedSlotManager.allocateSlotsAccordingTo(FineGrainedSlotManager.java:722)
> ~[classes/:?]
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.FineGrainedSlotManager.checkResourceRequirements(FineGrainedSlotManager.java:645)
> ~[classes/:?]
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.FineGrainedSlotManager.lambda$null$12(FineGrainedSlotManager.java:603)
> ~[classes/:?]
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> [?:1.8.0_392]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> [?:1.8.0_392]
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> [?:1.8.0_392]
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> [?:1.8.0_392]
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> [?:1.8.0_392]
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> [?:1.8.0_392]
> at java.lang.Thread.run(Thread.java:750) [?:1.8.0_392]
> Caused by: java.util.concurrent.RejectedExecutionException: Task
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@4bbc0b10
> rejected from
> java.util.concurrent.ScheduledThreadPoolExecutor@7a45cd9a[Shutting down, pool
> size = 1, active threads = 1, queued tasks = 1, completed tasks = 194]
> at
> java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2063)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:326)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:533)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.ScheduledThreadPoolExecutor.execute(ScheduledThreadPoolExecutor.java:622)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.Executors$DelegatedExecutorService.execute(Executors.java:668)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.CompletableFuture$UniCompletion.claim(CompletableFuture.java:543)
> ~[?:1.8.0_392]
> at
> java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:826)
> ~[?:1.8.0_392]
> ... 14 more
> {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)