You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Piotr Nowojski (Jira)" <ji...@apache.org> on 2019/11/28 14:50:00 UTC
[jira] [Updated] (FLINK-10377) Remove precondition in
TwoPhaseCommitSinkFunction.notifyCheckpointComplete
[ https://issues.apache.org/jira/browse/FLINK-10377?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Piotr Nowojski updated FLINK-10377:
-----------------------------------
Description:
The precondition {{checkState(pendingTransactionIterator.hasNext(), "checkpoint completed, but no transaction pending");}} in {{TwoPhaseCommitSinkFunction.notifyCheckpointComplete()}} seems too strict, because checkpoints can overtake checkpoints and will fail the precondition. In this case the commit was already performed by the first notification and subsumes the late checkpoint. I think the check can be removed.
edit:
As [reported by a user on the user mailing list|http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/UNCHECKED-Error-while-confirming-Checkpoint-td23213.html], {{TwoPhaseCommitSinkFunction#notifyCheckpointComplete}} can fail with the following exception:
{noformat}
java.lang.RuntimeException: Error while confirming checkpoint
at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:1205)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
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)
Caused by: java.lang.IllegalStateException: checkpoint completed, but no transaction pending
at org.apache.flink.util.Preconditions.checkState(Preconditions.java:195)
at org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction.notifyCheckpointComplete(TwoPhaseCommitSinkFunction.java:267)
at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.notifyCheckpointComplete(AbstractUdfStreamOperator.java:130)
at org.apache.flink.streaming.runtime.tasks.StreamTask.notifyCheckpointComplete(StreamTask.java:822)
at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:1200)
... 5 more
{noformat}
This can happen in the following scenario:
# savepoint is triggered
# checkpoint is triggered
# checkpoint completes (but it doesn't subsume the savepoint, because checkpoints subsume only other checkpoints).
# savepoint completes
In this case, {{TwoPhaseCommitSinkFunction}} receives first notification that the later checkpoint completed, it commits both savepoint and the checkpoint. Later when savepoint {{notifyCheckpointComplete}} arrives, the above error will occur.
Possible trivial fix is to remove that failing {{checkState}}.
was:The precondition {{checkState(pendingTransactionIterator.hasNext(), "checkpoint completed, but no transaction pending");}} in {{TwoPhaseCommitSinkFunction.notifyCheckpointComplete()}} seems too strict, because checkpoints can overtake checkpoints and will fail the precondition. In this case the commit was already performed by the first notification and subsumes the late checkpoint. I think the check can be removed.
> Remove precondition in TwoPhaseCommitSinkFunction.notifyCheckpointComplete
> --------------------------------------------------------------------------
>
> Key: FLINK-10377
> URL: https://issues.apache.org/jira/browse/FLINK-10377
> Project: Flink
> Issue Type: Bug
> Components: Connectors / Common
> Affects Versions: 1.5.0, 1.6.0
> Reporter: Stefan Richter
> Assignee: Stefan Richter
> Priority: Major
> Labels: pull-request-available
> Time Spent: 10m
> Remaining Estimate: 0h
>
> The precondition {{checkState(pendingTransactionIterator.hasNext(), "checkpoint completed, but no transaction pending");}} in {{TwoPhaseCommitSinkFunction.notifyCheckpointComplete()}} seems too strict, because checkpoints can overtake checkpoints and will fail the precondition. In this case the commit was already performed by the first notification and subsumes the late checkpoint. I think the check can be removed.
> edit:
> As [reported by a user on the user mailing list|http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/UNCHECKED-Error-while-confirming-Checkpoint-td23213.html], {{TwoPhaseCommitSinkFunction#notifyCheckpointComplete}} can fail with the following exception:
> {noformat}
> java.lang.RuntimeException: Error while confirming checkpoint
> at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:1205)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 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)
> Caused by: java.lang.IllegalStateException: checkpoint completed, but no transaction pending
> at org.apache.flink.util.Preconditions.checkState(Preconditions.java:195)
> at org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction.notifyCheckpointComplete(TwoPhaseCommitSinkFunction.java:267)
> at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.notifyCheckpointComplete(AbstractUdfStreamOperator.java:130)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.notifyCheckpointComplete(StreamTask.java:822)
> at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:1200)
> ... 5 more
> {noformat}
> This can happen in the following scenario:
> # savepoint is triggered
> # checkpoint is triggered
> # checkpoint completes (but it doesn't subsume the savepoint, because checkpoints subsume only other checkpoints).
> # savepoint completes
> In this case, {{TwoPhaseCommitSinkFunction}} receives first notification that the later checkpoint completed, it commits both savepoint and the checkpoint. Later when savepoint {{notifyCheckpointComplete}} arrives, the above error will occur.
> Possible trivial fix is to remove that failing {{checkState}}.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)