[
https://issues.apache.org/jira/browse/FLINK-3594?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15189899#comment-15189899
]
Stephan Ewen commented on FLINK-3594:
-------------------------------------
Chesnay almost pinned it down precisely.
The actual error is that Tasks can enter a checkpoint immediately before they
are done, and then set {{isRunning}} to false while the checkpoint is in
progress.
That leads the checkpoint method to throw a {{CancelTaskException}} which is
caught in the asynchronous checkpoint trigger, causing to mark the task as
failed.
> StreamTask may fail when checkpoint is concurrent to regular termination
> ------------------------------------------------------------------------
>
> Key: FLINK-3594
> URL: https://issues.apache.org/jira/browse/FLINK-3594
> Project: Flink
> Issue Type: Bug
> Reporter: Chesnay Schepler
> Assignee: Stephan Ewen
> Priority: Critical
> Labels: test-stability
>
> Some tests in the KafkaConsumerTestBase rely on throwing a SuccessException
> to stop the streaming job if the test condition is fulfilled.
> The job then fails, and it is checked whether the cause was a
> SuccessException. if so, the test is marked as a success, otherwise as a
> failure.
> However, should this exception be thrown while a checkpoint is being
> triggered, the exception that stop the job is not the SuccessException, but a
> CancelTaskException.
> This should affect every test that uses the SuccessException.
> observed here: https://travis-ci.org/apache/flink/jobs/114523189
> The problem is that the exception causes the StreamTask to enter the finally
> block inside invoke(), which sets isRunning to false. Within
> triggerCheckpoint() isRunning is then checked for being false, and if so a
> CancelTaskException is thrown.
> This seems like an issue of the runtime; i observed other tests failing,
> without giving a good cause since the CancelTaskException masks it.
> I was wondering whether triggerCheckpoint() could return false instead of
> throwing an exception, and simply assume that an exception will be thrown
> within invoke().
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)