Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-24846

AsyncWaitOperator fails during stop-with-savepoint

    XMLWordPrintableJSON

Details

    Description

      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) ~[?:?]
      
      

      As reported by a user on the mailing list:

      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

      Attachments

        1. log-jm.txt
          412 kB
          Piotr Nowojski

        Activity

          People

            akalashnikov Anton Kalashnikov
            pnowojski Piotr Nowojski
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: