yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r928509364


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java:
##########
@@ -91,27 +130,133 @@ public CompletableFuture<Acknowledge> 
sendEvent(OperatorEvent evt) {
                                                         new 
FlinkException(msg, failure)));
                             }
                         },
-                        sendingExecutor);
+                        mainThreadExecutor);
 
-        sendingExecutor.execute(
+        mainThreadExecutor.execute(
                 () -> {
-                    sender.sendEvent(sendAction, sendResult);
+                    sendEventInternal(sendAction, sendResult);
                     
incompleteFuturesTracker.trackFutureWhileIncomplete(result);
                 });
+
         return result;
     }
 
-    @Override
-    public ExecutionAttemptID getExecution() {
-        return subtaskAccess.currentAttempt();
+    private void sendEventInternal(
+            Callable<CompletableFuture<Acknowledge>> sendAction,
+            CompletableFuture<Acknowledge> result) {
+        checkRunsInMainThread();
+
+        if (isClosed) {
+            blockedEvents.add(new BlockedEvent(sendAction, result));
+        } else {
+            callSendAction(sendAction, result);
+        }
     }
 
-    @Override
-    public int getSubtask() {
-        return subtaskAccess.getSubtaskIndex();
+    private void callSendAction(
+            Callable<CompletableFuture<Acknowledge>> sendAction,
+            CompletableFuture<Acknowledge> result) {
+        try {
+            final CompletableFuture<Acknowledge> sendResult = 
sendAction.call();
+            FutureUtils.forward(sendResult, result);
+        } catch (Throwable t) {
+            ExceptionUtils.rethrowIfFatalError(t);
+            result.completeExceptionally(t);
+        }
     }
 
-    private boolean isReady() {
-        return subtaskAccess.hasSwitchedToRunning().isDone();
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the 
checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple 
coordinator
+     * checkpoints would be attempted overlapping, which is currently not 
supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to 
identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by 
finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != 
checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for 
checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId 
= %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked 
until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is 
incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (checkpointId == currentCheckpointId) {
+            isClosed = true;
+            return true;
+        }
+        return false;
+    }
+
+    boolean isClosed() {
+        checkRunsInMainThread();
+        return isClosed;
+    }
+
+    void openGatewayAndUnmarkCheckpoint(long expectedCheckpointId) {
+        checkRunsInMainThread();
+
+        if (expectedCheckpointId != currentCheckpointId) {

Review Comment:
   According to our offline discussion, the OC can receive the 
`AcknowledgeCheckpointEvent` of a previous checkpoint when it has closed 
gateways for a newer checkpoint, so in this case, the subtask gateway should 
ignore events about the previous checkpoint. I'll add the throw Exception code 
if `expectedCheckpointId > currentCheckpointId && currentCheckpointId != 
NO_CHECKPOINT`.



-- 
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