wangyang0918 commented on a change in pull request #13644: URL: https://github.com/apache/flink/pull/13644#discussion_r510123914
########## File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionService.java ########## @@ -0,0 +1,219 @@ +/* + * 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.kubernetes.kubeclient.FlinkKubeClient; +import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesWatch; +import org.apache.flink.kubernetes.utils.KubernetesUtils; +import org.apache.flink.runtime.leaderelection.AbstractLeaderElectionService; +import org.apache.flink.runtime.leaderelection.LeaderContender; +import org.apache.flink.util.function.FunctionUtils; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY; +import static org.apache.flink.kubernetes.utils.Constants.LEADER_ADDRESS_KEY; +import static org.apache.flink.kubernetes.utils.Constants.LEADER_SESSION_ID_KEY; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Leader election service for multiple JobManagers. The active JobManager is elected using Kubernetes. + * The current leader's address as well as its leader session ID is published via Kubernetes ConfigMap. + * Note that the contending lock and leader storage are using the same ConfigMap. And every component(e.g. + * ResourceManager, Dispatcher, RestEndpoint, JobManager for each job) will have a separate ConfigMap. + */ +public class KubernetesLeaderElectionService extends AbstractLeaderElectionService { + + private final FlinkKubeClient kubeClient; + + private final Executor executor; + + private final String configMapName; + + private final KubernetesLeaderElector leaderElector; + + private KubernetesWatch kubernetesWatch; + + // Labels will be used to clean up the ha related ConfigMaps. + private Map<String, String> configMapLabels; + + KubernetesLeaderElectionService( + FlinkKubeClient kubeClient, + Executor executor, + KubernetesLeaderElectionConfiguration leaderConfig) { + + this.kubeClient = checkNotNull(kubeClient, "Kubernetes client should not be null."); + this.executor = checkNotNull(executor, "Executor should not be null."); + this.configMapName = leaderConfig.getConfigMapName(); + this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl()); + this.leaderContender = null; + this.configMapLabels = KubernetesUtils.getConfigMapLabels( + leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY); + } + + @Override + public void internalStart(LeaderContender contender) { + CompletableFuture.runAsync(leaderElector::run, executor); + kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl()); + } + + @Override + public void internalStop() { + if (kubernetesWatch != null) { + kubernetesWatch.close(); + } + } + + @Override + protected void writeLeaderInformation() { + try { + kubeClient.checkAndUpdateConfigMap( + configMapName, + configMap -> { + if (leaderElector.hasLeadership(configMap)) { + // Get the updated ConfigMap with new leader information + if (confirmedLeaderAddress != null && confirmedLeaderSessionID != null) { + configMap.getData().put(LEADER_ADDRESS_KEY, confirmedLeaderAddress); + configMap.getData().put(LEADER_SESSION_ID_KEY, confirmedLeaderSessionID.toString()); + } + configMap.getLabels().putAll(configMapLabels); + return Optional.of(configMap); + } + return Optional.empty(); + }).get(); + } catch (Exception e) { + leaderContender.handleError(new Exception("Could not update ConfigMap " + configMapName, e)); + } + } + + @Override + protected boolean hasLeadership() { + return kubeClient.getConfigMap(configMapName) + .map(leaderElector::hasLeadership) + .orElse(false); + } + + @Override + public String toString() { + return "KubernetesLeaderElectionService{configMapName='" + configMapName + "'}"; + } + + private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler { + + @Override + public void isLeader() { + onGrantLeadership(); + } + + @Override + public void notLeader() { + // Clear the leader information in ConfigMap + try { + kubeClient.checkAndUpdateConfigMap( + configMapName, + configMap -> { + // Do not need to check the leader here + configMap.getData().remove(LEADER_ADDRESS_KEY); + configMap.getData().remove(LEADER_SESSION_ID_KEY); + return Optional.of(configMap); + } + ).get(); + } catch (Exception e) { + leaderContender.handleError( + new Exception("Could not remove leader information from ConfigMap " + configMapName, e)); + } + onRevokeLeadership(); + // Continue to contend the leader + CompletableFuture.runAsync(leaderElector::run, executor); + } + } + + private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> { + + @Override + public void onAdded(List<KubernetesConfigMap> configMaps) { + // noop + } + + @Override + public void onModified(List<KubernetesConfigMap> configMaps) { + if (hasLeadership()) { + configMaps.forEach(configMap -> { + if (isLeaderUpdatedExternally(configMap)) { + if (configMap.getName().equals(configMapName)) { + // the data field does not correspond to the expected leader information + if (logger.isDebugEnabled()) { + logger.debug("Correcting leader information in {} by {}.", + configMapName, leaderContender.getDescription()); + } + writeLeaderInformation(); + } else { + logger.warn("Ignoring the modified event since it does not belong to {}.", this); + } + } + }); + } + } + + @Override + public void onDeleted(List<KubernetesConfigMap> configMaps) { Review comment: Your concern is valid. In such case, we could not recover the old state. So I will throw a error in the first version when the ConfigMap is deleted or updated externally. ########## File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java ########## @@ -104,6 +106,67 @@ KubernetesWatch watchPodsAndDoCallback( Map<String, String> labels, WatchCallbackHandler<KubernetesPod> podCallbackHandler); + /** + * Create the ConfigMap with specified content. If the ConfigMap already exists, a FlinkRuntimeException will be + * thrown. + * + * @param configMap ConfigMap. + * + * @return Return the ConfigMap create future. + */ + CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap); + + /** + * Get the ConfigMap with specified name. + * + * @param name ConfigMap name. + * + * @return Return the ConfigMap, or empty if the ConfigMap does not exist. + */ + Optional<KubernetesConfigMap> getConfigMap(String name); + + /** + * Update an existing ConfigMap with the data. Benefit from <a href=https://kubernetes.io/docs/reference/using-api/api-concepts/#resource-versions> + * resource version</a> and combined with {@link #getConfigMap(String)}, we could perform a get-check-and-update + * transactional operation. Since concurrent modification could happen on a same ConfigMap, + * the update operation may fail. We need to retry internally. The max retry attempts could be + * configured via {@link org.apache.flink.kubernetes.configuration.KubernetesConfigOptions#KUBERNETES_TRANSACTIONAL_OPERATION_MAX_RETRIES}. + * + * @param configMapName ConfigMap to be replaced with. + * @param function Function to be applied to the obtained ConfigMap and get a new updated one. If the returned Review comment: Yes. For example, if we want to discard a checkpoint, we need to remove the data from ConfigMap. Also we do the `stateHandle.discardState()` just like following. Right? ``` kubeClient.checkAndUpdateConfigMap( configMapName, configMap -> { if (KubernetesLeaderElector.hasLeadership(configMap)) { final String content = configMap.getData().remove(key); if (content != null) { final RetrievableStateHandle<T> stateHandle = deserializeObject(content); stateHandle.discardState(); } return Optional.of(configMap); } return Optional.empty(); }).get(); ``` ########## File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalServiceTest.java ########## @@ -0,0 +1,89 @@ +/* + * 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.kubernetes.kubeclient.FlinkKubeClient; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap; +import org.apache.flink.kubernetes.utils.Constants; + +import org.junit.Test; + +import java.util.Collections; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertThat; + +/** + * Tests for the {@link KubernetesLeaderRetrievalService}. + */ +public class KubernetesLeaderRetrievalServiceTest extends KubernetesHighAvailabilityTestBase { Review comment: Yeah. I am adding the integration tests. ---------------------------------------------------------------- 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: us...@infra.apache.org