Bom Dia Vinicius, Can You still find (and post) the exception stack from your jobmanager log, the flink client log does not reveal enough information. Your situation reminds me of something similar I had. In the log you might find something like this or similar:
2022-03-07 02:15:41,347 INFO org.apache.flink.runtime.jobmaster.JobMaster [] - Triggering stop-with-savepoint for job e12f22653f79194863ab426312dd666a. 2022-03-07 02:15:41,380 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Triggering checkpoint 4983974 (type=SAVEPOINT_SUSPEND) @ 1646615741347 for job e12f22653f79194863ab426312dd666a. 2022-03-07 02:15:43,042 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Decline checkpoint 4983974 by task 0e659ac720e3e0b3e4072dbc1cc85cd3 of job e12f22653f79194863ab426312dd666a at container_e1093_1646358077201_0002_01_000001 @ ulxxphaddtn02.adgr.net (dataPort=44767). org.apache.flink.util.SerializedThrowable: Asynchronous task checkpoint failed. at org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.handleExecutionException(AsyncCheckpointRunnable.java:279) ~[flink-dist_2.11-1.13.0.jar:1.13.0] BTW what Flink version are you running? What is EMR (what technology underneath). From: Vinicius Peracini <vinicius.perac...@zenvia.com> Sent: Montag, 7. März 2022 20:46 To: Dawid Wysakowicz <dwysakow...@apache.org> Cc: user@flink.apache.org Subject: Re: Could not stop job with a savepoint Hi Dawid, thanks for the reply. The job was still in progress and producing events. Unfortunately I was not able to stop the job with a savepoint or to just create a savepoint. I had to stop the job without the savepoint and restore the state using the last checkpoint. Still reviewing my configuration and trying to figure out why this is happening. Any help would be appreciated. Thanks! On Mon, Mar 7, 2022 at 11:56 AM Dawid Wysakowicz <dwysakow...@apache.org<mailto:dwysakow...@apache.org>> wrote: Hi, From the exception it seems the job has been already done when you're triggering the savepoint. Best, Dawid On 07/03/2022 14:56, Vinicius Peracini wrote: Hello everyone, I have a Flink job (version 1.14.0 running on EMR) and I'm having this issue while trying to stop a job with a savepoint on S3: org.apache.flink.util.FlinkException: Could not stop with a savepoint job "df3a3c590fabac737a17f1160c21094c". 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 java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1730) at org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41) at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1132) Caused by: java.util.concurrent.ExecutionException: java.util.concurrent.CompletionException: org.apache.flink.runtime.checkpoint.CheckpointException: Checkpoint Coordinator is suspending. at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357) at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928) at org.apache.flink.client.cli.CliFrontend.lambda$stop$5(CliFrontend.java:579) ... 9 more I'm using incremental and unaligned checkpoints (aligned checkpoint timeout is 30 seconds). I also tried to create the savepoint without stopping the job (using flink savepoint command) and got the same error. Any idea what is happening here? Thanks in advance, Aviso Legal: Este documento pode conter informações confidenciais e/ou privilegiadas. Se você não for o destinatário ou a pessoa autorizada a receber este documento, não deve usar, copiar ou divulgar as informações nele contidas ou tomar qualquer ação baseada nessas informações. Disclaimer: The information contained in this document may be privileged and confidential and protected from disclosure. If the reader of this document is not the intended recipient, or an employee agent responsible for delivering this document to the intended recipient, you are hereby notified that any dissemination, distribution or copying of this communication is strictly prohibited. Aviso Legal: Este documento pode conter informações confidenciais e/ou privilegiadas. Se você não for o destinatário ou a pessoa autorizada a receber este documento, não deve usar, copiar ou divulgar as informações nele contidas ou tomar qualquer ação baseada nessas informações. Disclaimer: The information contained in this document may be privileged and confidential and protected from disclosure. If the reader of this document is not the intended recipient, or an employee agent responsible for delivering this document to the intended recipient, you are hereby notified that any dissemination, distribution or copying of this communication is strictly prohibited. Diese Nachricht ist ausschliesslich für den Adressaten bestimmt und beinhaltet unter Umständen vertrauliche Mitteilungen. Da die Vertraulichkeit von e-Mail-Nachrichten nicht gewährleistet werden kann, übernehmen wir keine Haftung für die Gewährung der Vertraulichkeit und Unversehrtheit dieser Mitteilung. Bei irrtümlicher Zustellung bitten wir Sie um Benachrichtigung per e-Mail und um Löschung dieser Nachricht sowie eventueller Anhänge. Jegliche unberechtigte Verwendung oder Verbreitung dieser Informationen ist streng verboten. This message is intended only for the named recipient and may contain confidential or privileged information. As the confidentiality of email communication cannot be guaranteed, we do not accept any responsibility for the confidentiality and the intactness of this message. If you have received it in error, please advise the sender by return e-mail and delete this message and any attachments. Any unauthorised use or dissemination of this information is strictly prohibited.