Piotr Nowojski created FLINK-24846:
--------------------------------------

             Summary: AsyncWaitOperator fails during stop-with-savepoint
                 Key: FLINK-24846
                 URL: https://issues.apache.org/jira/browse/FLINK-24846
             Project: Flink
          Issue Type: Bug
          Components: Runtime / Task
            Reporter: Piotr Nowojski
         Attachments: log-jm.txt

{noformat}
Caused by: 
org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox$MailboxClosedException:
 Mailbox is in state QUIESCED, but is required to be in state OPEN for put 
operations.
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.checkPutStateConditions(TaskMailboxImpl.java:269)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.put(TaskMailboxImpl.java:197)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl.execute(MailboxExecutorImpl.java:74)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.api.common.operators.MailboxExecutor.execute(MailboxExecutor.java:103)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.api.operators.async.AsyncWaitOperator.outputCompletedElement(AsyncWaitOperator.java:304)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.api.operators.async.AsyncWaitOperator.access$100(AsyncWaitOperator.java:78)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.api.operators.async.AsyncWaitOperator$ResultHandler.processResults(AsyncWaitOperator.java:370)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.api.operators.async.AsyncWaitOperator$ResultHandler.lambda$processInMailbox$0(AsyncWaitOperator.java:351)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90) 
~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.drain(MailboxProcessor.java:177)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.afterInvoke(StreamTask.java:854)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:767) 
~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958)
 ~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at 
org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:937) 
~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:766) 
~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575) 
~[flink-dist_2.11-1.14.0.jar:1.14.0]
        at java.lang.Thread.run(Thread.java:829) ~[?:?]

{noformat}

As reported by a user on [the mailing 
list:|https://mail-archives.apache.org/mod_mbox/flink-user/202111.mbox/%3CCAO6dnLwtLNxkr9qXG202ysrnse18Wgvph4hqHZe3ar8cuXAfDw%40mail.gmail.com%3E]
{quote}
I failed to stop a job with savepoint with the following message:
Inconsistent execution state after stopping with savepoint. At least one 
execution is still in one of the following states: FAILED, CANCELED. A global 
fail-over is triggered to recover the job 452594f3ec5797f399e07f95c884a44b.

The job manager said
 A savepoint was created at 
hdfs://mobdata-flink-hdfs/driving-habits/svpts/savepoint-452594-f60305755d0e 
but the corresponding job 452594f3ec5797f399e07f95c884a44b didn't terminate 
successfully.
while complaining about
Mailbox is in state QUIESCED, but is required to be in state OPEN for put 
operations.

Is it okay to ignore this kind of error?

Please see the attached files for the detailed context.

FYI, 
- I used the latest 1.14.0
- I started the job with "$FLINK_HOME"/bin/flink run --target yarn-per-job
- I couldn't reproduce the exception using the same jar so I might not able to 
provide DUBUG messages
{quote}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to