StephanEwen commented on a change in pull request #11274: 
[FLINK-16177][checkpointing] Integrate OperatorCoordinator checkpoint 
triggering and committing
URL: https://github.com/apache/flink/pull/11274#discussion_r387532703
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java
 ##########
 @@ -0,0 +1,141 @@
+/*
+ * 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.checkpoint;
+
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.Executor;
+
+/**
+ * All the logic related to taking checkpoints of the {@link 
OperatorCoordinator}s.
+ *
+ * <p>NOTE: This class has a simplified error handling logic. If one of the 
several coordinator checkpoints
+ * fail, no cleanup is triggered for the other concurrent ones. That is okay, 
since they all produce just byte[]
+ * as the result. We have to change that once we allow then to create external 
resources that actually need
+ * to be cleaned up.
+ */
+final class OperatorCoordinatorCheckpoints {
+
+       public static CompletableFuture<CoordinatorSnapshot> 
triggerCoordinatorCheckpoint(
+                       final OperatorCoordinatorCheckpointContext 
coordinatorInfo,
+                       final long checkpointId) throws Exception {
+
+               final CompletableFuture<byte[]> checkpointFuture =
+                               
coordinatorInfo.coordinator().checkpointCoordinator(checkpointId);
+
+               return checkpointFuture.thenApply(
+                               (state) -> new CoordinatorSnapshot(
+                                               coordinatorInfo, new 
ByteStreamStateHandle(coordinatorInfo.operatorId().toString(), state))
+               );
+       }
+
+       public static CompletableFuture<AllCoordinatorSnapshots> 
triggerAllCoordinatorCheckpoints(
+                       final Collection<OperatorCoordinatorCheckpointContext> 
coordinators,
+                       final long checkpointId) throws Exception {
+
+               final Collection<CompletableFuture<CoordinatorSnapshot>> 
individualSnapshots = new ArrayList<>(coordinators.size());
+
+               for (final OperatorCoordinatorCheckpointContext coordinator : 
coordinators) {
+                       
individualSnapshots.add(triggerCoordinatorCheckpoint(coordinator, 
checkpointId));
+               }
+
+               return 
FutureUtils.combineAll(individualSnapshots).thenApply(AllCoordinatorSnapshots::new);
+       }
+
+       public static CompletableFuture<Void> 
triggerAndAcknowledgeAllCoordinatorCheckpoints(
+                       final Collection<OperatorCoordinatorCheckpointContext> 
coordinators,
+                       final PendingCheckpoint checkpoint,
+                       final Executor acknowledgeExecutor) throws Exception {
+
+               final CompletableFuture<AllCoordinatorSnapshots> snapshots =
+                               triggerAllCoordinatorCheckpoints(coordinators, 
checkpoint.getCheckpointId());
+
+               return snapshots
+                               .thenAcceptAsync(
+                                               (allSnapshots) -> 
acknowledgeAllCoordinators(checkpoint, allSnapshots.snapshots),
+                                               acknowledgeExecutor);
+       }
+
+       public static CompletableFuture<Void> 
triggerAndAcknowledgeAllCoordinatorCheckpointsWithCompletion(
+                       final Collection<OperatorCoordinatorCheckpointContext> 
coordinators,
+                       final PendingCheckpoint checkpoint,
+                       final Executor acknowledgeExecutor) throws 
CompletionException {
+
+               try {
+                       return 
triggerAndAcknowledgeAllCoordinatorCheckpoints(coordinators, checkpoint, 
acknowledgeExecutor);
+               } catch (Exception e) {
+                       throw new CompletionException(e);
+               }
+       }
+
+       // 
------------------------------------------------------------------------
+
+       private static void acknowledgeAllCoordinators(PendingCheckpoint 
checkpoint, Collection<CoordinatorSnapshot> snapshots) {
+               for (final CoordinatorSnapshot snapshot : snapshots) {
+                       
checkpoint.acknowledgeCoordinatorState(snapshot.coordinator, snapshot.state);
 
 Review comment:
   True, we can check for anything `!= SUCCESS` and exit early with an 
exception.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to