[ 
https://issues.apache.org/jira/browse/FLINK-10377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16622196#comment-16622196
 ] 

ASF GitHub Bot commented on FLINK-10377:
----------------------------------------

StefanRRichter opened a new pull request #6723: [FLINK-10377] Remove 
precondition in TwoPhaseCommitSinkFunction.notif…
URL: https://github.com/apache/flink/pull/6723
 
 
   …yCheckpointComplete
   
   ## What is the purpose of the change
   The precondition `checkState(pendingTransactionIterator.hasNext(), 
"checkpoint completed, but no transaction pending");` in 
`TwoPhaseCommitSinkFunction.notifyCheckpointComplete()` seems to 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.
   
   ## Brief change log
   Removed the precondition without replacements and created a test case for 
ou-of-order notification.
   
   ## Verifying this change
   
   `TwoPhaseCommitSinkFunctionTest#testSubsumedNotificationOfPreviousCheckpoint`
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove precondition in TwoPhaseCommitSinkFunction.notifyCheckpointComplete
> --------------------------------------------------------------------------
>
>                 Key: FLINK-10377
>                 URL: https://issues.apache.org/jira/browse/FLINK-10377
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming Connectors
>    Affects Versions: 1.5.0, 1.6.0
>            Reporter: Stefan Richter
>            Assignee: Stefan Richter
>            Priority: Major
>              Labels: pull-request-available
>
> The precondition {{checkState(pendingTransactionIterator.hasNext(), 
> "checkpoint completed, but no transaction pending");}} in 
> {{TwoPhaseCommitSinkFunction.notifyCheckpointComplete()}} seems to 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.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to