lindong28 commented on code in PR #20454:
URL: https://github.com/apache/flink/pull/20454#discussion_r945428399


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java:
##########
@@ -86,6 +91,24 @@ class SubtaskGatewayImpl implements 
OperatorCoordinator.SubtaskGateway {
 
     @Override
     public CompletableFuture<Acknowledge> sendEvent(OperatorEvent evt) {
+        return sendEventWithCallBackOnCompletion(
+                evt,
+                (success, failure) -> {
+                    if (failure != null && subtaskAccess.isStillRunning()) {

Review Comment:
   Should we use `failure instanceof TaskNotRunningException` to check whether 
the task is running?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -299,9 +374,73 @@ public void resetToCheckpoint(long checkpointId, @Nullable 
byte[] checkpointData
     }
 
     private void checkpointCoordinatorInternal(
-            final long checkpointId, final CompletableFuture<byte[]> result) {
+            long checkpointId, CompletableFuture<byte[]> result) {
         mainThreadExecutor.assertRunningInMainThread();
 
+        try {
+            if (currentPendingCheckpointId != OperatorCoordinator.NO_CHECKPOINT
+                    && currentPendingCheckpointId != checkpointId) {
+                throw new IllegalStateException(
+                        String.format(
+                                "Cannot checkpoint coordinator for checkpoint 
%d, "
+                                        + "since checkpoint %d has already 
started.",
+                                checkpointId, currentPendingCheckpointId));
+            }
+
+            if (latestAttemptedCheckpointId >= checkpointId) {
+                throw new IllegalStateException(
+                        String.format(
+                                "Regressing checkpoint IDs. Previous 
checkpointId = %d, new checkpointId = %d",
+                                latestAttemptedCheckpointId, checkpointId));
+            }
+
+            subtaskGatewayMap.forEach(
+                    (subtask, gateway) -> 
gateway.markForCheckpoint(checkpointId));
+
+            
Preconditions.checkState(acknowledgeCloseGatewayFutureMap.isEmpty());
+        } catch (Throwable t) {
+            result.completeExceptionally(t);
+            globalFailureHandler.handleGlobalFailure(t);
+            return;
+        }
+
+        currentPendingCheckpointId = checkpointId;
+        latestAttemptedCheckpointId = checkpointId;
+
+        for (int subtask : subtaskGatewayMap.keySet()) {
+            acknowledgeCloseGatewayFutureMap.put(subtask, new 
CompletableFuture<>());
+            final OperatorEvent closeGatewayEvent = new 
CloseGatewayEvent(checkpointId, subtask);
+            subtaskGatewayMap
+                    .get(subtask)
+                    .sendEventWithCallBackOnCompletion(
+                            closeGatewayEvent,
+                            (success, failure) -> {
+                                if (failure != null) {
+                                    // If the close gateway event failed to 
reach the subtask for
+                                    // some reason, the coordinator would 
trigger a fail-over on
+                                    // the subtask if the subtask is still 
running. If the subtask
+                                    // has been not running, it is guaranteed 
that the coordinator
+                                    // won't receive more events from this 
subtask before the
+                                    // current checkpoint finishes, which is 
equivalent to receiving
+                                    // ACK from this subtask.
+                                    if (!(failure instanceof 
TaskNotRunningException)) {
+                                        subtaskGatewayMap
+                                                .get(subtask)
+                                                
.triggerTaskFailover(closeGatewayEvent, failure);

Review Comment:
   Would it be simpler to move this logic to 
`SubtaskGatewayImpl::sendEventWithCallBackOnCompletion(...)`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -299,9 +374,73 @@ public void resetToCheckpoint(long checkpointId, @Nullable 
byte[] checkpointData
     }
 
     private void checkpointCoordinatorInternal(
-            final long checkpointId, final CompletableFuture<byte[]> result) {
+            long checkpointId, CompletableFuture<byte[]> result) {
         mainThreadExecutor.assertRunningInMainThread();
 
+        try {
+            if (currentPendingCheckpointId != OperatorCoordinator.NO_CHECKPOINT

Review Comment:
   Would it be simpler to use `if (currentPendingCheckpointId != 
OperatorCoordinator.NO_CHECKPOINT)` here?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseGatewayEvent.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.operators.coordination;
+
+import java.util.Objects;
+
+/**
+ * An {@link OperatorEvent} sent from an {@link OperatorCoordinator} to its 
subtask to signal that
+ * the communication gateway needs to be temporarily closed for a specific 
checkpoint.
+ */
+public class CloseGatewayEvent implements OperatorEvent {
+
+    /** The ID of the checkpoint that this event is related to. */
+    private final long checkpointId;
+
+    /** The index of the subtask that this event is related to. */
+    private final int subtaskIndex;

Review Comment:
   Can we remove this variable?



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to