[
https://issues.apache.org/jira/browse/FLINK-12858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16882944#comment-16882944
]
Kostas Kloudas edited comment on FLINK-12858 at 7/11/19 1:21 PM:
-----------------------------------------------------------------
I think that the problem here is not limited to the {{stop-with-savepoint}}.
This seems to be related to that the {{BarrierBuffer.checkSizeLimit()}}
configuration parameter aborts a checkpoint and reports it to the checkpoint
coordinator without respecting the user-specified parameter that specifies the
policy of how a checkpoint failure should be handled (the
{{CheckpointExceptionHandler}}) and without somehow informing the task about it.
Another implication of that is that I am not sure how this also affects the
already existing {{cancel-with-savepoint}} command from a user's perspective.
was (Author: kkl0u):
I think that the problem here is not the {{stop-with-savepoint}} but the fact
that the {{BarrierBuffer.checkSizeLimit()}} configuration parameter aborts a
checkpoint and reports it to the checkpoint coordinator without respecting the
user-specified parameter that specifies the policy of how a checkpoint failure
should be handled. which is the {{CheckpointExceptionHandler}}.
For example I am not sure how this also affects the already existing
{{cancel-with-savepoint}} command.
> Potentially not properly working Flink job in case of stop-with-savepoint
> failure
> ---------------------------------------------------------------------------------
>
> Key: FLINK-12858
> URL: https://issues.apache.org/jira/browse/FLINK-12858
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Checkpointing
> Affects Versions: 1.9.0
> Reporter: Alex
> Assignee: Alex
> Priority: Blocker
>
> Current implementation of stop-with-savepoint (FLINK-11458) would lock the
> thread (on {{syncSavepointLatch}}) that carries
> {{StreamTask.performCheckpoint()}}. For non-source tasks, this thread is
> implied to be the task's main thread (stop-with-savepoint deliberately stops
> any activity in the task's main thread).
> Unlocking happens either when the task is cancelled or when the corresponding
> checkpoint is acknowledged.
> It's possible, that other downstream tasks of the same Flink job "soft" fail
> the checkpoint/savepoint due to various reasons (for example, due to max
> buffered bytes {{BarrierBuffer.checkSizeLimit()}}. In such case, the
> checkpoint abortion would be notified to JM . But it looks like, the
> checkpoint coordinator would handle such abortion as usual and assume that
> the Flink job continues running.
--
This message was sent by Atlassian JIRA
(v7.6.14#76016)