tillrohrmann commented on a change in pull request #13871:
URL: https://github.com/apache/flink/pull/13871#discussion_r518897446



##########
File path: 
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.kubernetes.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
+
+import java.util.concurrent.Executor;
+import java.util.function.Function;
+
+import static 
org.apache.flink.kubernetes.utils.KubernetesUtils.createCompletedCheckpointStore;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Factory to create {@link CompletedCheckpointStore} and {@link 
CheckpointIDCounter}.
+ */
+public class KubernetesCheckpointRecoveryFactory implements 
CheckpointRecoveryFactory {
+
+       private final FlinkKubeClient kubeClient;
+
+       private final Executor executor;
+
+       // Function to get the ConfigMap name for checkpoint. Input is job id, 
and output is ConfigMap name.
+       private final Function<JobID, String> getConfigMapNameFunction;
+
+       private final Configuration configuration;
+
+       private final String lockIdentity;
+
+       /**
+        * Create a KubernetesCheckpointRecoveryFactory.
+        *
+        * @param kubeClient Kubernetes client
+        * @param configuration Flink configuration
+        * @param executor IO executor to run blocking calls
+        * @param function Function to get the ConfigMap name for checkpoint.
+        * @param lockIdentity Lock identity of current HA service
+        */
+       public KubernetesCheckpointRecoveryFactory(
+                       FlinkKubeClient kubeClient,
+                       Configuration configuration,
+                       Executor executor,
+                       Function<JobID, String> function,
+                       String lockIdentity) {
+
+               this.kubeClient = checkNotNull(kubeClient);
+               this.configuration = checkNotNull(configuration);
+               this.executor = checkNotNull(executor);
+               this.getConfigMapNameFunction = checkNotNull(function);
+               this.lockIdentity = checkNotNull(lockIdentity);
+       }
+
+       @Override
+       public CompletedCheckpointStore createCheckpointStore(
+                       JobID jobID,
+                       int maxNumberOfCheckpointsToRetain,
+                       ClassLoader userClassLoader) throws Exception {
+
+               final String configMapName = 
getConfigMapNameFunction.apply(jobID);
+               return createCompletedCheckpointStore(

Review comment:
       nit: I am usually a bit against static method imports because it makes 
it harder to see where this method comes from. I think it is better to also put 
the class name of the method so that it is easily visible.

##########
File path: 
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounter.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.kubernetes.highavailability;
+
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
+import 
org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector;
+import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static 
org.apache.flink.kubernetes.utils.Constants.CHECKPOINT_COUNTER_KEY;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link CheckpointIDCounter} implementation for Kubernetes. The counter will 
be stored in
+ * JobManager-{@link org.apache.flink.api.common.JobID}-leader ConfigMap. The 
key is
+ * {@link org.apache.flink.kubernetes.utils.Constants#CHECKPOINT_COUNTER_KEY},
+ * and value is counter value.
+ */
+public class KubernetesCheckpointIDCounter implements CheckpointIDCounter {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(KubernetesCheckpointIDCounter.class);
+
+       private final FlinkKubeClient kubeClient;
+
+       private final String configMapName;
+
+       private final String lockIdentity;
+
+       private boolean running;
+
+       public KubernetesCheckpointIDCounter(FlinkKubeClient kubeClient, String 
configMapName, String lockIdentity) {
+               this.kubeClient = checkNotNull(kubeClient);
+               this.configMapName = checkNotNull(configMapName);
+               this.lockIdentity = checkNotNull(lockIdentity);
+
+               this.running = false;
+       }
+
+       @Override
+       public void start() {
+               if (!running) {
+                       running = true;
+               }
+       }
+
+       @Override
+       public void shutdown(JobStatus jobStatus) {
+               if (!running) {
+                       return;
+               }
+               running = false;
+
+               LOG.info("Shutting down.");
+               if (jobStatus.isGloballyTerminalState()) {
+                       LOG.info("Removing counter from ConfigMap {}", 
configMapName);
+                       kubeClient.checkAndUpdateConfigMap(
+                               configMapName,
+                               configMap -> {
+                                       if 
(KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+                                               
configMap.getData().remove(CHECKPOINT_COUNTER_KEY);
+                                               return Optional.of(configMap);
+                                       }
+                                       return Optional.empty();
+                               });
+               }
+       }
+
+       @Override
+       public long getAndIncrement() throws Exception {
+               final AtomicLong current = new AtomicLong();
+               final boolean updated = kubeClient.checkAndUpdateConfigMap(
+                       configMapName,
+                       configMap -> {
+                               if 
(KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+                                       final long currentValue = 
getCurrentCounter(configMap);
+                                       current.set(currentValue);
+                                       
configMap.getData().put(CHECKPOINT_COUNTER_KEY, String.valueOf(currentValue + 
1));
+                                       return Optional.of(configMap);
+                               }
+                               return Optional.empty();
+                       }
+               ).get();

Review comment:
       nit: line break could make it easier to read the code.




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


Reply via email to