[
https://issues.apache.org/jira/browse/FLINK-30337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17691499#comment-17691499
]
Alexey Novakov edited comment on FLINK-30337 at 2/21/23 10:16 AM:
------------------------------------------------------------------
I have been able to reproduce this issue as well.
Flink CLI Stop command fails with above exception after timeout and there is
one more exception in *statefun-worker* pod:
{code:java}
2023-02-21 10:06:36,818 INFO
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable [] -
feedback-union -> functions (1/1)#0 - asynchronous part of checkpoint 2 could
not be completed.
java.util.concurrent.CancellationException: null
at java.util.concurrent.FutureTask.report(Unknown Source) ~[?:?]
at java.util.concurrent.FutureTask.get(Unknown Source) ~[?:?]
at
org.apache.flink.util.concurrent.FutureUtils.runIfNotDoneAndGet(FutureUtils.java:645)
~[flink-dist_2.12-1.14.3.jar:1.14.3]
at
org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:57)
~[flink-dist_2.12-1.14.3.jar:1.14.3]
at
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.finalizeNonFinishedSnapshots(AsyncCheckpointRunnable.java:177)
~[flink-dist_2.12-1.14.3.jar:1.14.3]
at
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.run(AsyncCheckpointRunnable.java:124)
[flink-dist_2.12-1.14.3.jar:1.14.3]
at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) [?:?]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) [?:?]
at java.lang.Thread.run(Unknown Source) [?:?] {code}
was (Author: novakov.alex):
I have been able to reproduce this issue as well.
Flink CLI Stop command fails with above exception after timeout and there is
one more exception in *statefun-worker* pod:
{code:java}
2023-02-21 10:06:36,818 INFO
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable [] -
feedback-union -> functions (1/1)#0 - asynchronous part of checkpoint 2 could
not be completed.
java.util.concurrent.CancellationException: null
at java.util.concurrent.FutureTask.report(Unknown Source) ~[?:?]
at java.util.concurrent.FutureTask.get(Unknown Source) ~[?:?]
at
org.apache.flink.util.concurrent.FutureUtils.runIfNotDoneAndGet(FutureUtils.java:645)
~[flink-dist_2.12-1.14.3.jar:1.14.3]
at
org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:57)
~[flink-dist_2.12-1.14.3.jar:1.14.3]
at
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.finalizeNonFinishedSnapshots(AsyncCheckpointRunnable.java:177)
~[flink-dist_2.12-1.14.3.jar:1.14.3]
at
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.run(AsyncCheckpointRunnable.java:124)
[flink-dist_2.12-1.14.3.jar:1.14.3]
at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) [?:?]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) [?:?]
at java.lang.Thread.run(Unknown Source) [?:?] {code}
> Stateful Functions application throws an exception when stopping a job
> gracefully creating a final savepoint
> ------------------------------------------------------------------------------------------------------------
>
> Key: FLINK-30337
> URL: https://issues.apache.org/jira/browse/FLINK-30337
> Project: Flink
> Issue Type: Bug
> Components: Stateful Functions
> Affects Versions: statefun-3.2.0
> Reporter: Ali Bahadir Zeybek
> Priority: Minor
>
> When running a Stateful Functions applications, if the stop[1] command is
> executed, the client throws a FlinkException with the following stack trace
> where *953498833da99ec437758b49b7d5befd* is the specific job id:
>
> {code:java}
> The program finished with the following
> exception:org.apache.flink.util.FlinkException: Could not stop with a
> savepoint job "953498833da99ec437758b49b7d5befd".
> at
> org.apache.flink.client.cli.CliFrontend.lambda$stop$5(CliFrontend.java:581)
> at
> org.apache.flink.client.cli.CliFrontend.runClusterAction(CliFrontend.java:1002)
> at org.apache.flink.client.cli.CliFrontend.stop(CliFrontend.java:569)
> at
> org.apache.flink.client.cli.CliFrontend.parseAndRun(CliFrontend.java:1069)
> at
> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1132)
> at
> org.apache.flink.runtime.security.contexts.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:28)
> at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1132)
> Caused by: java.util.concurrent.TimeoutException
> at java.base/java.util.concurrent.CompletableFuture.timedGet(Unknown
> Source)
> at java.base/java.util.concurrent.CompletableFuture.get(Unknown Source)
> at
> org.apache.flink.client.cli.CliFrontend.lambda$stop$5(CliFrontend.java:579)
> ... 6 more {code}
>
> How to reproduce:
> # Follow the README[2] of the k8s deployment example of the
> *flink-statefun-playground* project to have a running application
> # Open the Flink UI that is started to get the *JOB_ID*
> # Detect the *STATEFUN_MASTER_POD_NAME* by running: *kubectl get pods
> --namespace statefun*
> # Start a shell into the *statefun-master* pod by issuing the: *kubectl exec
> -it --namespace statefun $STATEFUN_MASTER_POD_NAME – /bin/bash*
> # Run the stop command: *./bin/flink stop --savepointPath
> /tmp/flink-savepoints $JOB_ID*
>
> [1]:
> [https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/cli/#stopping-a-job-gracefully-creating-a-final-savepoint]
> [2]:
> [https://github.com/apache/flink-statefun-playground/blob/main/deployments/k8s/README.md]
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)