AHeise commented on a change in pull request #19138:
URL: https://github.com/apache/flink/pull/19138#discussion_r830864828



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java
##########
@@ -159,22 +207,54 @@ protected void advanceToEndOfEventTime() {
         output.emitWatermark(Watermark.MAX_WATERMARK);
     }
 
+    @Override
+    protected void declineCheckpoint(long checkpointId) {
+        cleanupCheckpoint(checkpointId);
+        super.declineCheckpoint(checkpointId);
+    }
+
+    @Override
+    public Future<Void> notifyCheckpointAbortAsync(
+            long checkpointId, long latestCompletedCheckpointId) {
+        cleanupCheckpoint(checkpointId);
+        return super.notifyCheckpointAbortAsync(checkpointId, 
latestCompletedCheckpointId);
+    }
+
+    @Override
+    public Future<Void> notifyCheckpointSubsumedAsync(long checkpointId) {
+        cleanupCheckpoint(checkpointId);
+        return super.notifyCheckpointSubsumedAsync(checkpointId);
+    }
+
     // --------------------------
 
     private void triggerCheckpointForExternallyInducedSource(long 
checkpointId) {
-        final CheckpointOptions checkpointOptions =
-                CheckpointOptions.forConfig(
-                        CheckpointType.CHECKPOINT,
-                        CheckpointStorageLocationReference.getDefault(),
-                        configuration.isExactlyOnceCheckpointMode(),
-                        configuration.isUnalignedCheckpointsEnabled(),
-                        
configuration.getAlignedCheckpointTimeout().toMillis());
-        final long timestamp = System.currentTimeMillis();
+        UntriggeredCheckpoint untriggeredCheckpoint = 
untriggeredCheckpoints.remove(checkpointId);
+        if (untriggeredCheckpoint != null) {
+            cleanupOldCheckpoints(checkpointId);
+            // common case: RPC before external sources induces it
+            triggerCheckpointNowAsync(
+                    untriggeredCheckpoint.getMetadata(),
+                    untriggeredCheckpoint.getCheckpointOptions());
+        } else {
+            // rare case: external source induced first
+            triggeredCheckpoints.add(checkpointId);
+        }
+    }
 
-        final CheckpointMetaData checkpointMetaData =
-                new CheckpointMetaData(checkpointId, timestamp, timestamp);
+    /**
+     * Cleanup any orphaned checkpoint before the given currently triggered 
checkpoint. These
+     * checkpoint may occur when the checkpoint is cancelled but the RPC is 
lost.
+     */
+    private void cleanupOldCheckpoints(long checkpointId) {
+        triggeredCheckpoints.headSet(checkpointId).clear();
+        untriggeredCheckpoints.headMap(checkpointId).clear();
+    }
 
-        super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions);
+    /** Remove temporary data about a canceled checkpoint. */
+    private void cleanupCheckpoint(long checkpointId) {

Review comment:
       Good catch. I keep forgetting that RPC are not in mailbox. :/
   I added assertions now and called them over mailbox in the async RPC calls. 
(`declineCheckpoint` is already in mailbox).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to