[ 
https://issues.apache.org/jira/browse/FLINK-14544?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16961208#comment-16961208
 ] 

Congxian Qiu(klion26) edited comment on FLINK-14544 at 10/28/19 4:47 PM:
-------------------------------------------------------------------------

>From the given log,  If I understand right, this test failed because the log 
>contains exception of {{MailboxStateException. we can filter out the 
>"}}{{MailboxStateException" before counting the exception counts. and need to 
>figure out why the MailboxStateException throws out.}}

 
{code:java}
Running externalized checkpoints test, with ORIGINAL_DOP=2 NEW_DOP=2 and 
STATE_BACKEND_TYPE=file STATE_BACKEND_FILE_ASYNC=true STATE_BACKEND_ROCKSDB_INC 
     REMENTAL=false SIMULATE_FAILURE=true ...^M
19106 Waiting for job (2d7c274d4561078c592df0bbb1dfad52) to reach terminal 
state FAILED ...^M
19107 Job (2d7c274d4561078c592df0bbb1dfad52) reached terminal state FAILED^M
19108 Restoring job with externalized checkpoint at 
/home/travis/build/apache/flink/flink-end-to-end-tests/test-scripts/temp-test-directory-50753031208/extern
      alized-chckpt-e2e-backend-dir/2d7c274d4561078c592df0bbb1dfad52/chk-2 ...^M
19109 Job (824b849f432dcffdeb0d18ab6b1f7d6c) is running.^M
19110 Checking for errors...^M
19111 Found error in log files:^M
......
25083 2019-10-25 20:46:09,361 ERROR org.apache.flink.runtime.taskmanager.Task   
                  - Error while canceling task FailureMapper (1/1).^M
25084 java.util.concurrent.RejectedExecutionException: 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxStateException: Mailbox 
is in state CLOSED, but       is required to be in state OPEN for put 
operations.^M
25085         at 
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorImpl.executeFirst(MailboxExecutorImpl.java:75)^M
25086         at 
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor.sendPriorityLetter(MailboxProcessor.java:176)^M
25087         at 
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor.allActionsCompleted(MailboxProcessor.java:172)^M
25088         at 
org.apache.flink.streaming.runtime.tasks.StreamTask.cancel(StreamTask.java:540)^M
25089         at 
org.apache.flink.runtime.taskmanager.Task.cancelInvokable(Task.java:1191)^M
25090         at 
org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:764)^M
25091         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:521)^M
25092         at java.lang.Thread.run(Thread.java:748)^M
25093 Caused by: 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxStateException: Mailbox 
is in state CLOSED, but is required to be in state OPEN for p      ut 
operations.^M
25094         at 
org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.checkPutStateConditions(TaskMailboxImpl.java:199)^M
25095         at 
org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.putHeadInternal(TaskMailboxImpl.java:141)^M
25096         at 
org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.putFirst(TaskMailboxImpl.java:131)^M
25097         at 
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorImpl.executeFirst(MailboxExecutorImpl.java:73)^M
25098         ... 7 more^M
{code}


was (Author: klion26):
>From the given log,  If I understand right, this test failed because the log 
>contains exception of {{MailboxStateException. we can filter out the 
>"}}{{MailboxStateException" before counting the exception counts.}}

 
{code:java}
Running externalized checkpoints test, with ORIGINAL_DOP=2 NEW_DOP=2 and 
STATE_BACKEND_TYPE=file STATE_BACKEND_FILE_ASYNC=true STATE_BACKEND_ROCKSDB_INC 
     REMENTAL=false SIMULATE_FAILURE=true ...^M
19106 Waiting for job (2d7c274d4561078c592df0bbb1dfad52) to reach terminal 
state FAILED ...^M
19107 Job (2d7c274d4561078c592df0bbb1dfad52) reached terminal state FAILED^M
19108 Restoring job with externalized checkpoint at 
/home/travis/build/apache/flink/flink-end-to-end-tests/test-scripts/temp-test-directory-50753031208/extern
      alized-chckpt-e2e-backend-dir/2d7c274d4561078c592df0bbb1dfad52/chk-2 ...^M
19109 Job (824b849f432dcffdeb0d18ab6b1f7d6c) is running.^M
19110 Checking for errors...^M
19111 Found error in log files:^M
......
25083 2019-10-25 20:46:09,361 ERROR org.apache.flink.runtime.taskmanager.Task   
                  - Error while canceling task FailureMapper (1/1).^M
25084 java.util.concurrent.RejectedExecutionException: 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxStateException: Mailbox 
is in state CLOSED, but       is required to be in state OPEN for put 
operations.^M
25085         at 
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorImpl.executeFirst(MailboxExecutorImpl.java:75)^M
25086         at 
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor.sendPriorityLetter(MailboxProcessor.java:176)^M
25087         at 
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor.allActionsCompleted(MailboxProcessor.java:172)^M
25088         at 
org.apache.flink.streaming.runtime.tasks.StreamTask.cancel(StreamTask.java:540)^M
25089         at 
org.apache.flink.runtime.taskmanager.Task.cancelInvokable(Task.java:1191)^M
25090         at 
org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:764)^M
25091         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:521)^M
25092         at java.lang.Thread.run(Thread.java:748)^M
25093 Caused by: 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxStateException: Mailbox 
is in state CLOSED, but is required to be in state OPEN for p      ut 
operations.^M
25094         at 
org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.checkPutStateConditions(TaskMailboxImpl.java:199)^M
25095         at 
org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.putHeadInternal(TaskMailboxImpl.java:141)^M
25096         at 
org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.putFirst(TaskMailboxImpl.java:131)^M
25097         at 
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorImpl.executeFirst(MailboxExecutorImpl.java:73)^M
25098         ... 7 more^M
{code}

> Resuming Externalized Checkpoint after terminal failure (file, async) 
> end-to-end test fails on travis
> -----------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-14544
>                 URL: https://issues.apache.org/jira/browse/FLINK-14544
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing, Tests
>    Affects Versions: 1.10.0
>            Reporter: Yu Li
>            Priority: Blocker
>              Labels: test-stability
>
> From the log we could see below error message and then the job was terminated 
> due to job exceeded the maximum log length. 
> {code}
> 2019-10-25 20:46:09,361 ERROR org.apache.flink.runtime.taskmanager.Task       
>               - Error while canceling task FailureMapper (1/1).
> java.util.concurrent.RejectedExecutionException: 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxStateException: 
> Mailbox is in state CLOSED, but is required to be in state OPEN for put 
> operations.
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorImpl.executeFirst(MailboxExecutorImpl.java:75)
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor.sendPriorityLetter(MailboxProcessor.java:176)
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor.allActionsCompleted(MailboxProcessor.java:172)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.cancel(StreamTask.java:540)
>       at 
> org.apache.flink.runtime.taskmanager.Task.cancelInvokable(Task.java:1191)
>       at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:764)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:521)
>       at java.lang.Thread.run(Thread.java:748)
> Caused by: 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxStateException: 
> Mailbox is in state CLOSED, but is required to be in state OPEN for put 
> operations.
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.checkPutStateConditions(TaskMailboxImpl.java:199)
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.putHeadInternal(TaskMailboxImpl.java:141)
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.putFirst(TaskMailboxImpl.java:131)
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorImpl.executeFirst(MailboxExecutorImpl.java:73)
>       ... 7 more
> {code}
> https://api.travis-ci.org/v3/job/602788586/log.txt



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to