You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/10/12 08:19:00 UTC

[jira] [Commented] (FLINK-10377) Remove precondition in TwoPhaseCommitSinkFunction.notifyCheckpointComplete

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

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

StefanRRichter closed pull request #6723: [FLINK-10377] Remove precondition in TwoPhaseCommitSinkFunction.notif…
URL: https://github.com/apache/flink/pull/6723
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
index 2ffb6d5810e..0e7be9b2e46 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
@@ -255,7 +255,6 @@ public final void notifyCheckpointComplete(long checkpointId) throws Exception {
 		//
 
 		Iterator<Map.Entry<Long, TransactionHolder<TXN>>> pendingTransactionIterator = pendingCommitTransactions.entrySet().iterator();
-		checkState(pendingTransactionIterator.hasNext(), "checkpoint completed, but no transaction pending");
 
 		while (pendingTransactionIterator.hasNext()) {
 			Map.Entry<Long, TransactionHolder<TXN>> entry = pendingTransactionIterator.next();
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java
index 2970b87789d..b97eb0c43e1 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java
@@ -138,6 +138,22 @@ private void closeTestHarness() throws Exception {
 		harness.close();
 	}
 
+	@Test
+	public void testSubsumedNotificationOfPreviousCheckpoint() throws Exception {
+		harness.open();
+		harness.processElement("42", 0);
+		harness.snapshot(0, 1);
+		harness.processElement("43", 2);
+		harness.snapshot(1, 3);
+		harness.processElement("44", 4);
+		harness.snapshot(2, 5);
+		harness.notifyOfCompletedCheckpoint(2);
+		harness.notifyOfCompletedCheckpoint(1);
+
+		assertExactlyOnce(Arrays.asList("42", "43", "44"));
+		assertEquals(1, tmpDirectory.listFiles().size()); // one for currentTransaction
+	}
+
 	@Test
 	public void testNotifyOfCompletedCheckpoint() throws Exception {
 		harness.open();


 

----------------------------------------------------------------
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:
users@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 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.



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