[
https://issues.apache.org/jira/browse/FLINK-14544?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16961638#comment-16961638
]
Congxian Qiu(klion26) commented on FLINK-14544:
-----------------------------------------------
from the log, found that the {{MailboxStateException}} throws out after
{{Artificial failure in the second job.}}
{code:java}
2019-10-25 20:46:09,345 INFO org.apache.flink.runtime.taskmanager.Task
- FailureMapper (1/1) (466747dfea13738afd021da649dc53f4) switched
from RUNNING to FAILED.
java.lang.Exception: Artificial failure.
at
org.apache.flink.streaming.tests.FailureMapper.map(FailureMapper.java:59)
at
org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:41)
at
org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:173)
at
org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.processElement(StreamTaskNetworkInput.java:151)
at
org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.emitNext(StreamTaskNetworkInput.java:128)
at
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:69)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:280)
at
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:152)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:423)
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:696)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:521)
at java.lang.Thread.run(Thread.java:748)
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}
so currently, the conclusion is that, the test `Resuming Externalized
Checkpoint after terminal failure (file, async) ` complete checkpoint in job 1,
restore from checkpoint , and complete more checkpoint in job2 , but the log
contains {{MailboxStateException}}, so we see the test failed.
* the first job complete checkpoints {{2019-10-25 20:46:08,614 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed
checkpoint 2 for job 2d7c274d4561078c592df0bbb1dfad52 (156791 bytes in 367
ms).}}
* trigger artifical exception
* retore from the checkpoint completed by the previous job
** 2019-10-25 20:46:13,358 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Starting job
824b849f432dcffdeb0d18ab6b1f7d6c from savepoint
file:///home/travis/build/apache/flink/flink-end-to-end-tests/test-scripts/temp-test-directory-50753031208/externalized-chckpt-e2e-backend-dir/2d7c274d4561078c592df0bbb1dfad52/chk-2
()
2019-10-25 20:46:13,378 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Reset the
checkpoint ID of job 824b849f432dcffdeb0d18ab6b1f7d6c to 3.
2019-10-25 20:46:13,378 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Restoring job
824b849f432dcffdeb0d18ab6b1f7d6c from latest valid checkpoint: Checkpoint 2 @ 0
for 824b849f432dcffdeb0d18ab6b1f7d6c.
* complete more new checkpoints in new job
** 2019-10-25 20:46:15,608 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed
checkpoint 3 for job 824b849f432dcffdeb0d18ab6b1f7d6c (160262 bytes in 1154 ms).
2019-10-25 20:46:15,655 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering
checkpoint 4 @ 1572036375608 for job 824b849f432dcffdeb0d18ab6b1f7d6c.
2019-10-25 20:46:15,831 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed
checkpoint 4 for job 824b849f432dcffdeb0d18ab6b1f7d6c (226751 bytes in 221 ms).
* trigger {{Artificial failure}} and fail the job(the log above)
As the analysis, I think we can change the test as
* do not trigger artifical exception in the second job
* ignore the {{MailboxStateException}} when finding exception in log, I think
if there indeed an {{MailboxStateException}} need we attention, the test will
be failed in previous stages before finding exception/error in logs?
> 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)