[
https://issues.apache.org/jira/browse/FLINK-21215?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17276131#comment-17276131
]
Piotr Nowojski commented on FLINK-21215:
----------------------------------------
I believe this is caused by
{noformat}
org.apache.flink.util.SerializedThrowable: Asynchronous task checkpoint failed.
at
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.handleExecutionException(AsyncCheckpointRunnable.java:267)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.run(AsyncCheckpointRunnable.java:174)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
[?:1.8.0_275]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[?:1.8.0_275]
at java.lang.Thread.run(Thread.java:748) [?:1.8.0_275]
Caused by: org.apache.flink.util.SerializedThrowable: Could not materialize
checkpoint 17 for operator keyed (4/5)#5.
at
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.handleExecutionException(AsyncCheckpointRunnable.java:255)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
... 4 more
Caused by: org.apache.flink.util.SerializedThrowable:
org.apache.flink.runtime.checkpoint.CheckpointException: Checkpoint was
declined because one input stream is finished
at
java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
~[?:1.8.0_275]
at
java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
~[?:1.8.0_275]
at
org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:66)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.run(AsyncCheckpointRunnable.java:127)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
... 3 more
Caused by: org.apache.flink.util.SerializedThrowable: Checkpoint was declined
because one input stream is finished
at
org.apache.flink.streaming.runtime.io.checkpointing.SingleCheckpointBarrierHandler.abortInternal(SingleCheckpointBarrierHandler.java:249)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.io.checkpointing.SingleCheckpointBarrierHandler.processEndOfPartition(SingleCheckpointBarrierHandler.java:273)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.handleEvent(CheckpointedInputGate.java:186)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:158)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.emitNext(StreamTaskNetworkInput.java:180)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:406)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:190)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:615)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:579)
~[flink-streaming-java_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:763)
~[flink-runtime_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:565)
~[flink-runtime_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
... 1 more
{noformat}
Everything starts when task declines checkpoint, because: "Checkpoint was
declined because one input stream is finished", which is expected.
Unfortunately this is detected by AsyncChekpointRunnable and is wrapped into :
{code:java}
new CheckpointException(
CheckpointFailureReason.CHECKPOINT_ASYNC_EXCEPTION,
checkpointException));
{code}
This is then passed to the {{CheckpointCoordinator}} and incorrectly bumping
the tolerable checkpoint failures counter.
> UnalignedCheckpointITCase.execute Failed
> ----------------------------------------
>
> Key: FLINK-21215
> URL: https://issues.apache.org/jira/browse/FLINK-21215
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Checkpointing
> Affects Versions: 1.13.0
> Reporter: Guowei Ma
> Assignee: Piotr Nowojski
> Priority: Major
> Labels: test-stability
>
> [https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=12691&view=logs&j=34f41360-6c0d-54d3-11a1-0292a2def1d9&t=2d56e022-1ace-542f-bf1a-b37dd63243f2&l=9146]
>
>
> ... 4 more
> Caused by: org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint
> tolerable failure threshold.
> at
> org.apache.flink.runtime.checkpoint.CheckpointFailureManager.handleCheckpointException(CheckpointFailureManager.java:98)
>
> at
> org.apache.flink.runtime.checkpoint.CheckpointFailureManager.handleTaskLevelCheckpointException(CheckpointFailureManager.java:84)
>
> at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1930)
>
> at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveDeclineMessage(CheckpointCoordinator.java:1007)
>
> at
> org.apache.flink.runtime.scheduler.SchedulerBase.lambda$declineCheckpoint$9(SchedulerBase.java:1009)
>
> at
> org.apache.flink.runtime.scheduler.SchedulerBase.lambda$processCheckpointCoordinatorMessage$10(SchedulerBase.java:1025)
>
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
> at java.lang.Thread.run(Thread.java:748)
>
>
>
>
>
>
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)