Details
-
Bug
-
Status: Closed
-
Critical
-
Resolution: Fixed
-
1.9.2, 1.10.0
-
Checkpoint timeouts will now be treated as normal checkpoint failures and checked against `setTolerableCheckpointFailureNumber(...)`.
Description
As described in point 2: https://issues.apache.org/jira/browse/FLINK-17327?focusedCommentId=17090576&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17090576
(copy of description from above linked comment):
The logic in how CheckpointCoordinator handles checkpoint timeouts is broken. In your qinjunjerry examples, your job should have failed after first checkpoint failure, but checkpoints were time outing on CheckpointCoordinator after 5 seconds, before FlinkKafkaProducer was detecting Kafka failure after 2 minutes. Those timeouts were not checked against setTolerableCheckpointFailureNumber(...) limit, so the job was keep going with many timed out checkpoints. Now funny thing happens: FlinkKafkaProducer detects Kafka failure. Funny thing is that it depends where the failure was detected:
a) on processing record? no problem, job will failover immediately once failure is detected (in this example after 2 minutes)
b) on checkpoint? heh, the failure is reported to CheckpointCoordinator and gets ignored, as PendingCheckpoint has already been discarded 2 minutes ago So theoretically the checkpoints can keep failing forever and the job will not restart automatically, unless something else fails.
Even more funny things can happen if we mix FLINK-17350 . or b) with intermittent external system failure. Sink reports an exception, transaction was lost/aborted, Sink is in failed state, but if there will be a happy coincidence that it manages to accept further records, this exception can be lost and all of the records in those failed checkpoints will be lost forever as well. In all of the examples that qinjunjerry posted it hasn't happened. FlinkKafkaProducer was not able to recover after the initial failure and it was keep throwing exceptions until the job finally failed (but much later then it should have). And that's not guaranteed anywhere.
Attachments
Issue Links
- causes
-
FLINK-17327 Kafka unavailability could cause Flink TM shutdown
- Closed
- is caused by
-
FLINK-12364 Introduce a CheckpointFailureManager to centralized manage checkpoint failure
- Closed
- is related to
-
FLINK-17350 StreamTask should always fail immediately on failures in synchronous part of a checkpoint
- Closed
- relates to
-
FLINK-17043 Putting more information into accounting when failing a job in FailureManager
- Open
- links to