You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/15 06:13:21 UTC

[GitHub] [flink] wangyang0918 opened a new pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

wangyang0918 opened a new pull request #13644:
URL: https://github.com/apache/flink/pull/13644


   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   This is the first PR of [FLIP-144](https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API), adding the `KubernetesLeaderElectionService` and `KubernetesLeaderRetrievalService`. You could find more design details in the FLIP doc.
   
   In this PR, we add some necessary interfaces for the ConfigMap operations. The most important one is `FlinkKubeClient#checkAndUpdateConfigMap`. Benefit from [resource version](https://kubernetes.io/docs/reference/using-api/api-concepts/#resource-versions) and combined with `FlinkKubeClient#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 ` org.apache.flink.kubernetes.configuration.KubernetesConfigOptions#KUBERNETES_MAX_RETRY_ATTEMPTS`.
   
   ## Brief change log
   
   * edad84f4f497a7a987a05bc41ceb3e75e58ec37a Use a more generic WatchCallbackHandler to replace PodCallbackHandler
   * 02cf691f7deda6cf128a0fbd2bb0cdf6baf7390e Add public interfaces for ConfigMap operations(e.g. create/get/update/delete)
   * 8f868c13655b9d11149bf7ba6b2803c3ffff1391 Extract some common logics from ZooKeeperLeaderElectionService to AbstractLeaderElectionService
   * c32eeb3b2a45a5bb349394f9651c2e54b80bbfc3 Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API
   
   
   
   ## Verifying this change
   
   * Add unit tests for Fabric8KubeClient new added interfaces
   * Add unit tests `KubernetesLeaderElectionServiceTest` and `KubernetesLeaderRetrievalServiceTest` for LeaderElection and LeaderRetrieval
   
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (**yes** / no / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (**yes** / no)
     - If yes, how is the feature documented? (will add doc in a separate ticket)
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e33c52394024e8f1312c096a4b612d53d8848b97 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653) 
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 991bdfa5427479f845abaf91caba0069011ee9e3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r511277348



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {

Review comment:
       The kubernetes ITCase will be introduced to cover some corner cases.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513210810



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionEventHandler.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.leaderelection;
+
+/**
+ * Interface which should be implemented to response to {@link LeaderInformation} changes in
+ * {@link LeaderElectionDriver}.
+ */
+public interface LeaderElectionEventHandler {
+
+	/**
+	 * Called by specific {@link LeaderElectionDriver} when the leadership is granted.
+	 */
+	void onGrantLeadership();
+
+	/**
+	 * Called by specific {@link LeaderElectionDriver} when the leadership is revoked.
+	 */
+	void onRevokeLeadership();
+
+	/**
+	 * Called by specific {@link LeaderElectionDriver} when the leader information is changed. Then the
+	 * {@link DefaultLeaderElectionService} could write the leader information again if necessary.
+	 * @param leaderInformation leader information which contains leader session id and leader address.

Review comment:
       Yes. This method should only be called when the driver still has the leadership. I will add a description here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r515221220



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/AbstractKubernetesWatcher.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.client.KubernetesClientException;
+import io.fabric8.kubernetes.client.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Watcher for resources in Kubernetes.
+ */
+public abstract class AbstractKubernetesWatcher<T extends HasMetadata, K extends KubernetesResource<T>> implements Watcher<T> {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final FlinkKubeClient.WatchCallbackHandler<K> callbackHandler;
+
+	AbstractKubernetesWatcher(FlinkKubeClient.WatchCallbackHandler<K> callbackHandler) {
+		this.callbackHandler = callbackHandler;
+	}
+
+	@Override
+	public void onClose(KubernetesClientException cause) {
+		// null means the watcher is closed by expected.

Review comment:
       nit: `null means the watcher is closed normally`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.leaderelection;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderContender} and
+ * {@link LeaderElectionEventHandler}.
+ */
+public class TestingLeaderBase {
+	// The queues will be offered by subclasses
+	protected final BlockingQueue<LeaderInformation> leaderQueue = new LinkedBlockingQueue<>();
+	protected final BlockingQueue<LeaderInformation> revokeQueue = new LinkedBlockingQueue<>();
+	protected final BlockingQueue<Throwable> errorQueue = new LinkedBlockingQueue<>();
+
+	private boolean isLeader = false;
+	private Throwable error;
+
+	public void waitForLeader(long timeout) throws Exception {
+		final LeaderInformation leader = leaderQueue.poll(timeout, TimeUnit.MILLISECONDS);
+		if (leader == null || leader.equals(LeaderInformation.empty())) {
+			throw new TimeoutException("Contender was not elected as the leader within " + timeout + "ms");

Review comment:
       technically if `leader == LeaderInformation.empty()` and we still have time, then we should retry and see whether a leader will be elected later.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java
##########
@@ -160,6 +161,39 @@ public void testRetryCancellation() throws Exception {
 		}
 	}
 
+	/**
+	 * Test that {@link FutureUtils#retry} should stop at non-retryable exception.
+	 */
+	@Test
+	public void testStopAtNonRetryableException() {
+		final int retries = 10;
+		final int notRetry = 3;
+		final AtomicInteger atomicInteger = new AtomicInteger(0);
+		final String notRetryExceptionMsg = "Non-retryable exception";
+		CompletableFuture<Boolean> retryFuture = FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> {
+					if (atomicInteger.incrementAndGet() == notRetry) {
+						// throw non-retryable exception
+						throw new CompletionException(new FlinkRuntimeException(notRetryExceptionMsg));
+					} else {
+						throw new CompletionException(new FlinkException("Test exception"));
+					}
+				},
+				TestingUtils.defaultExecutor()),
+			retries,
+			throwable -> ExceptionUtils.findThrowable(throwable, FlinkException.class).isPresent(),
+			TestingUtils.defaultExecutor());
+
+		try {
+			retryFuture.get();
+			fail("Exception should be thrown.");
+		} catch (Exception ex) {
+			assertThat(ExceptionUtils.findThrowableWithMessage(ex, notRetryExceptionMsg).isPresent(), is(true));

Review comment:
       could be written `assertThat(ex, FlinkMatchers.containsCause(expectedException))` with `expectedException = new FlinkRuntimeException(notRetryExceptionMsg)`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.leaderelection;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderContender} and
+ * {@link LeaderElectionEventHandler}.
+ */
+public class TestingLeaderBase {
+	// The queues will be offered by subclasses
+	protected final BlockingQueue<LeaderInformation> leaderQueue = new LinkedBlockingQueue<>();
+	protected final BlockingQueue<LeaderInformation> revokeQueue = new LinkedBlockingQueue<>();
+	protected final BlockingQueue<Throwable> errorQueue = new LinkedBlockingQueue<>();
+
+	private boolean isLeader = false;
+	private Throwable error;
+
+	public void waitForLeader(long timeout) throws Exception {
+		final LeaderInformation leader = leaderQueue.poll(timeout, TimeUnit.MILLISECONDS);
+		if (leader == null || leader.equals(LeaderInformation.empty())) {
+			throw new TimeoutException("Contender was not elected as the leader within " + timeout + "ms");
+		}
+		isLeader = true;
+	}
+
+	public void waitForRevokeLeader(long timeout) throws Exception {
+		final LeaderInformation revoke = revokeQueue.poll(timeout, TimeUnit.MILLISECONDS);
+		if (revoke == null || !revoke.equals(LeaderInformation.empty())) {
+			throw new TimeoutException("Contender was not revoked within " + timeout + "ms");

Review comment:
       Same for the `revokeQueue`. If we have time left, then it could still happen that the leader gets revoked.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderInformation.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.leaderelection;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * Information about leader including the confirmed leader session id and leader address.
+ */
+public class LeaderInformation implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	@Nullable
+	private final UUID leaderSessionID;
+
+	@Nullable
+	private final String leaderAddress;
+
+	private static final LeaderInformation EMPTY = new LeaderInformation(null, null);
+
+	private LeaderInformation(@Nullable UUID leaderSessionID, @Nullable String leaderAddress) {
+		this.leaderSessionID = leaderSessionID;
+		this.leaderAddress = leaderAddress;
+	}
+
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return leaderSessionID;
+	}
+
+	@Nullable
+	public String getLeaderAddress() {
+		return leaderAddress;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj == this) {
+			return true;
+		} else if (obj != null && obj.getClass() == LeaderInformation.class) {
+			final LeaderInformation that = (LeaderInformation) obj;
+			return Objects.equals(this.leaderSessionID, that.leaderSessionID) &&
+				Objects.equals(this.leaderAddress, that.leaderAddress);
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public int hashCode() {
+		int result = Objects.hashCode(leaderSessionID);
+		result = 31 * result + Objects.hashCode(leaderAddress);
+		return result;
+	}
+
+	public static LeaderInformation known(@Nullable UUID leaderSessionID, @Nullable String leaderAddress) {

Review comment:
       I think we should remove the `@Nullable` annotations here because it is a known leader information.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.leaderelection;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderContender} and
+ * {@link LeaderElectionEventHandler}.
+ */
+public class TestingLeaderBase {
+	// The queues will be offered by subclasses
+	protected final BlockingQueue<LeaderInformation> leaderQueue = new LinkedBlockingQueue<>();
+	protected final BlockingQueue<LeaderInformation> revokeQueue = new LinkedBlockingQueue<>();
+	protected final BlockingQueue<Throwable> errorQueue = new LinkedBlockingQueue<>();
+
+	private boolean isLeader = false;
+	private Throwable error;
+
+	public void waitForLeader(long timeout) throws Exception {
+		final LeaderInformation leader = leaderQueue.poll(timeout, TimeUnit.MILLISECONDS);
+		if (leader == null || leader.equals(LeaderInformation.empty())) {
+			throw new TimeoutException("Contender was not elected as the leader within " + timeout + "ms");
+		}
+		isLeader = true;
+	}
+
+	public void waitForRevokeLeader(long timeout) throws Exception {
+		final LeaderInformation revoke = revokeQueue.poll(timeout, TimeUnit.MILLISECONDS);
+		if (revoke == null || !revoke.equals(LeaderInformation.empty())) {

Review comment:
       What else than `LeaderInformation.empty()` will be stored in `revokeQueue`? Maybe we don't need a `LeaderInformation` here.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderRetrievalListener} and
+ * {@link org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler}.
+ */
+public class TestingRetrievalBase {
+
+	private final BlockingQueue<LeaderInformation> leaderQueue = new LinkedBlockingQueue<>();
+
+	private String address;
+	private String oldAddress;
+	private UUID leaderSessionID;
+	private Exception exception;
+
+	public String waitForNewLeader(long timeout) throws Exception {
+		final LeaderInformation leader = leaderQueue.poll(timeout, TimeUnit.MILLISECONDS);

Review comment:
       we might retry polling from here if we still have some time left.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory);
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		synchronized (lock) {
+			leaderContender = contender;
+			leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver(
+				this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription());
+			LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping DefaultLeaderElectionService.");
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.close();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+							"LeaderElectionService has already been stopped.", leaderSessionID);
+					}
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+							"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+					}
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			if (running) {
+				return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("hasLeadership is called after the service is stopped, returning false.");
+				}
+				return false;
+			}
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+		leaderElectionDriver.writeLeaderInformation(
+			LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress));
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+
+				leaderContender.grantLeadership(issuedLeaderSessionID);

Review comment:
       I think we shouldn't call `leaderContender` from under the `lock`. This could lead to deadlocks.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderRetrievalListener} and
+ * {@link org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler}.
+ */
+public class TestingRetrievalBase {
+
+	private final BlockingQueue<LeaderInformation> leaderQueue = new LinkedBlockingQueue<>();
+
+	private String address;
+	private String oldAddress;
+	private UUID leaderSessionID;
+	private Exception exception;
+
+	public String waitForNewLeader(long timeout) throws Exception {
+		final LeaderInformation leader = leaderQueue.poll(timeout, TimeUnit.MILLISECONDS);
+
+		if (exception != null) {
+			throw exception;
+		} else if (leader == null || leader.getLeaderAddress() == null ||
+			leader.getLeaderAddress().equals(oldAddress)) {
+			throw new TimeoutException("Listener was not notified about a new leader within " + timeout + "ms");
+		}
+
+		address = leader.getLeaderAddress();
+		leaderSessionID = leader.getLeaderSessionID();
+		oldAddress = leader.getLeaderAddress();
+
+		return address;
+	}
+
+	public void waitForEmptyLeaderInformation(long timeout) throws Exception {
+		final LeaderInformation leader = leaderQueue.poll(timeout, TimeUnit.MILLISECONDS);

Review comment:
       Before polling, we should check `exception` and throw it if it is not `null`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory);
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		synchronized (lock) {
+			leaderContender = contender;
+			leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver(
+				this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription());
+			LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping DefaultLeaderElectionService.");
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.close();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+							"LeaderElectionService has already been stopped.", leaderSessionID);
+					}
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+							"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+					}
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			if (running) {
+				return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("hasLeadership is called after the service is stopped, returning false.");
+				}
+				return false;
+			}
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+		leaderElectionDriver.writeLeaderInformation(
+			LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress));
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+
+				leaderContender.grantLeadership(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the grant leadership notification since the {} has " +
+						"already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onRevokeLeadership() {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+				}
+
+				issuedLeaderSessionID = null;
+				clearConfirmedLeaderInformation();
+
+				leaderContender.revokeLeadership();
+
+				// Clear the old leader information on the external storage
+				leaderElectionDriver.writeLeaderInformation(LeaderInformation.empty());
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the revoke leadership notification since the {} " +
+						"has already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onLeaderInformationChange(LeaderInformation leaderInformation) {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Leader node changed while {} is the leader with session ID {}.",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID);
+				}
+				if (confirmedLeaderSessionID != null) {
+					final UUID leaderSessionID = leaderInformation.getLeaderSessionID();
+					final String leaderAddress = leaderInformation.getLeaderAddress();
+					final LeaderInformation confirmedLeaderInfo = LeaderInformation.known(
+						confirmedLeaderSessionID, confirmedLeaderAddress);
+					if (leaderSessionID == null && leaderAddress == null) {

Review comment:
       Maybe introduce `LeaderInformation.isEmpty()` and replace this line with it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionDriver.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.leaderelection;
+
+/**
+ * A {@link LeaderElectionDriver} is responsible for performing the leader election and storing the leader information.
+ * All the leader internal state is guarded by lock in {@link LeaderElectionService}. Different driver
+ * implementations do not need to care about the lock. And it should use {@link LeaderElectionEventHandler}
+ * if it want to respond to the leader change events.
+ *
+ * <p><strong>Important</strong>: The {@link LeaderElectionDriver} could not guarantee that there is no
+ * {@link LeaderElectionEventHandler} callbacks happen after {@link #close()}.
+ */
+public interface LeaderElectionDriver {
+
+	/**
+	 * Write the current leader information to external persistent storage(e.g. Zookeeper, Kubernetes ConfigMap). This
+	 * is a blocking IO operation.
+	 *
+	 * @param leaderInformation current leader information. It could be {@link LeaderInformation#empty()}, which means
+	 * the caller want to clear the leader information on external storage.
+	 */
+	void writeLeaderInformation(LeaderInformation leaderInformation);

Review comment:
       Will this only happen if the driver has the leadership or does the user of this interface has to make sure of it?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory);
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		synchronized (lock) {
+			leaderContender = contender;
+			leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver(
+				this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription());
+			LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping DefaultLeaderElectionService.");
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.close();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+							"LeaderElectionService has already been stopped.", leaderSessionID);
+					}
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+							"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+					}
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			if (running) {
+				return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("hasLeadership is called after the service is stopped, returning false.");
+				}
+				return false;
+			}
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+		leaderElectionDriver.writeLeaderInformation(
+			LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress));
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+
+				leaderContender.grantLeadership(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the grant leadership notification since the {} has " +
+						"already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onRevokeLeadership() {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+				}
+
+				issuedLeaderSessionID = null;
+				clearConfirmedLeaderInformation();
+
+				leaderContender.revokeLeadership();

Review comment:
       Same here with the `lock`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve the leader information from
+ * different storage. The leader address as well as the current leader session ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService, LeaderRetrievalEventHandler {
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderRetrievalDriverFactory leaderRetrievalDriverFactory;
+
+	@GuardedBy("lock")
+	@Nullable
+	private String lastLeaderAddress;
+
+	@GuardedBy("lock")
+	@Nullable
+	private UUID lastLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	/** Listener which will be notified about leader changes. */
+	@GuardedBy("lock")
+	private volatile LeaderRetrievalListener leaderListener;
+
+	@GuardedBy("lock")
+	private LeaderRetrievalDriver leaderRetrievalDriver;
+
+	/**
+	 * Creates a default leader retrieval service with specified {@link LeaderRetrievalDriverFactory}.
+	 *
+	 * @param leaderRetrievalDriverFactory {@link LeaderRetrievalDriverFactory} used for creating
+	 * {@link LeaderRetrievalDriver}.
+	 */
+	public DefaultLeaderRetrievalService(LeaderRetrievalDriverFactory leaderRetrievalDriverFactory) {
+		this.leaderRetrievalDriverFactory = checkNotNull(leaderRetrievalDriverFactory);
+
+		this.lastLeaderAddress = null;
+		this.lastLeaderSessionID = null;
+
+		running = false;
+	}
+
+	@Override
+	public void start(LeaderRetrievalListener listener) throws Exception {
+		checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "DefaultLeaderRetrievalService can " +
+			"only be started once.");
+
+		synchronized (lock) {
+			leaderListener = listener;
+			leaderRetrievalDriver = leaderRetrievalDriverFactory.createLeaderRetrievalDriver(
+				this, new LeaderRetrievalFatalErrorHandler());
+			LOG.info("Starting DefaultLeaderRetrievalService with {}.", leaderRetrievalDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Stopping DefaultLeaderRetrievalService.");
+			leaderRetrievalDriver.close();
+		}
+	}
+
+	/**
+	 * Called by specific {@link LeaderRetrievalDriver} to notify leader address.
+	 * @param leaderInformation new notified leader information
+	 * address. The exception will be handled by leader listener.
+	 */
+	@Override
+	@GuardedBy("lock")
+	public void notifyLeaderAddress(LeaderInformation leaderInformation) {
+		final UUID newLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String newLeaderAddress = leaderInformation.getLeaderAddress();
+		synchronized (lock) {
+			if (running) {
+				if (!Objects.equals(newLeaderAddress, lastLeaderAddress) ||
+					!Objects.equals(newLeaderSessionID, lastLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						if (newLeaderAddress == null && newLeaderSessionID == null) {
+							LOG.debug("Leader information was lost: The listener will be notified accordingly.");
+						} else {
+							LOG.debug(
+								"New leader information: Leader={}, session ID={}.",
+								newLeaderAddress,
+								newLeaderSessionID);
+						}
+					}
+
+					lastLeaderAddress = newLeaderAddress;
+					lastLeaderSessionID = newLeaderSessionID;
+					leaderListener.notifyLeaderAddress(newLeaderAddress, newLeaderSessionID);

Review comment:
       running this call outside of the `lock` might be safer in order to avoid deadlocks.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				// grant leadership
+				testingLeaderElectionDriver.isLeader();
+
+				testingContender.waitForLeader(timeout);
+				assertThat(testingContender.getDescription(), is(TEST_URL));
+				assertThat(testingContender.getLeaderSessionID(), is(leaderElectionService.getLeaderSessionID()));
+				// Check the external storage
+				assertThat(
+					testingLeaderElectionDriver.getLeaderInformation(),
+					is(LeaderInformation.known(leaderElectionService.getLeaderSessionID(), TEST_URL)));
+
+				// revoke leadership
+				testingLeaderElectionDriver.notLeader();
+				testingContender.waitForRevokeLeader(timeout);
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+				assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue()));
+				// External storage should be cleared
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				// Leader information changed on external storage. It should be corrected.
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+
+				testingLeaderElectionDriver.leaderInformationChanged(
+					LeaderInformation.known(UUID.randomUUID(), "faulty-address"));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+			});
+		}};
+	}
+
+	@Test
+	public void testHasLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(true));
+				assertThat(leaderElectionService.hasLeadership(UUID.randomUUID()), is(false));
+
+				leaderElectionService.stop();
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(false));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedWhenNoConfirmedSessionID() throws Exception {

Review comment:
       Wouldn't the name `testLeaderInformationChangedIfNotBeingLeader` be a bitter more precise here?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve the leader information from
+ * different storage. The leader address as well as the current leader session ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService, LeaderRetrievalEventHandler {
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderRetrievalDriverFactory leaderRetrievalDriverFactory;
+
+	@GuardedBy("lock")
+	@Nullable
+	private String lastLeaderAddress;
+
+	@GuardedBy("lock")
+	@Nullable
+	private UUID lastLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	/** Listener which will be notified about leader changes. */
+	@GuardedBy("lock")
+	private volatile LeaderRetrievalListener leaderListener;
+
+	@GuardedBy("lock")
+	private LeaderRetrievalDriver leaderRetrievalDriver;
+
+	/**
+	 * Creates a default leader retrieval service with specified {@link LeaderRetrievalDriverFactory}.
+	 *
+	 * @param leaderRetrievalDriverFactory {@link LeaderRetrievalDriverFactory} used for creating
+	 * {@link LeaderRetrievalDriver}.
+	 */
+	public DefaultLeaderRetrievalService(LeaderRetrievalDriverFactory leaderRetrievalDriverFactory) {
+		this.leaderRetrievalDriverFactory = checkNotNull(leaderRetrievalDriverFactory);
+
+		this.lastLeaderAddress = null;
+		this.lastLeaderSessionID = null;
+
+		running = false;
+	}
+
+	@Override
+	public void start(LeaderRetrievalListener listener) throws Exception {
+		checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "DefaultLeaderRetrievalService can " +
+			"only be started once.");
+
+		synchronized (lock) {
+			leaderListener = listener;
+			leaderRetrievalDriver = leaderRetrievalDriverFactory.createLeaderRetrievalDriver(
+				this, new LeaderRetrievalFatalErrorHandler());
+			LOG.info("Starting DefaultLeaderRetrievalService with {}.", leaderRetrievalDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Stopping DefaultLeaderRetrievalService.");
+			leaderRetrievalDriver.close();
+		}
+	}
+
+	/**
+	 * Called by specific {@link LeaderRetrievalDriver} to notify leader address.
+	 * @param leaderInformation new notified leader information
+	 * address. The exception will be handled by leader listener.
+	 */
+	@Override
+	@GuardedBy("lock")
+	public void notifyLeaderAddress(LeaderInformation leaderInformation) {
+		final UUID newLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String newLeaderAddress = leaderInformation.getLeaderAddress();
+		synchronized (lock) {
+			if (running) {
+				if (!Objects.equals(newLeaderAddress, lastLeaderAddress) ||
+					!Objects.equals(newLeaderSessionID, lastLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						if (newLeaderAddress == null && newLeaderSessionID == null) {
+							LOG.debug("Leader information was lost: The listener will be notified accordingly.");
+						} else {
+							LOG.debug(
+								"New leader information: Leader={}, session ID={}.",
+								newLeaderAddress,
+								newLeaderSessionID);
+						}
+					}
+
+					lastLeaderAddress = newLeaderAddress;
+					lastLeaderSessionID = newLeaderSessionID;
+					leaderListener.notifyLeaderAddress(newLeaderAddress, newLeaderSessionID);
+				}
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring notification since the {} has already been closed.", leaderRetrievalDriver);
+				}
+			}
+		}
+	}
+
+	private class LeaderRetrievalFatalErrorHandler implements FatalErrorHandler {
+
+		@Override
+		public void onFatalError(Throwable throwable) {
+			synchronized (lock) {
+				if (running) {
+					if (throwable instanceof LeaderRetrievalException) {
+						leaderListener.handleError((LeaderRetrievalException) throwable);

Review comment:
       Same here with the `lock` and `leaderListener`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				// grant leadership
+				testingLeaderElectionDriver.isLeader();
+
+				testingContender.waitForLeader(timeout);
+				assertThat(testingContender.getDescription(), is(TEST_URL));
+				assertThat(testingContender.getLeaderSessionID(), is(leaderElectionService.getLeaderSessionID()));
+				// Check the external storage
+				assertThat(
+					testingLeaderElectionDriver.getLeaderInformation(),
+					is(LeaderInformation.known(leaderElectionService.getLeaderSessionID(), TEST_URL)));
+
+				// revoke leadership
+				testingLeaderElectionDriver.notLeader();
+				testingContender.waitForRevokeLeader(timeout);
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+				assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue()));
+				// External storage should be cleared
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				// Leader information changed on external storage. It should be corrected.
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+
+				testingLeaderElectionDriver.leaderInformationChanged(
+					LeaderInformation.known(UUID.randomUUID(), "faulty-address"));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+			});
+		}};
+	}
+
+	@Test
+	public void testHasLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(true));
+				assertThat(leaderElectionService.hasLeadership(UUID.randomUUID()), is(false));
+
+				leaderElectionService.stop();
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(false));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedWhenNoConfirmedSessionID() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation faultyLeader = LeaderInformation.known(UUID.randomUUID(), "faulty-address");
+				testingLeaderElectionDriver.leaderInformationChanged(faultyLeader);
+				// External storage should keep the wrong value.
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(faultyLeader));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnGrantLeadershipHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.isLeader();
+				// leader contender is not granted leadership
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnLeaderInformationChangeHappenAfterStop() throws Exception {

Review comment:
       Nit: `testOnLeaderInformationChangeHappensAfterStop`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderRetrievalServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testNotifyLeaderAddress() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation newLeader = LeaderInformation.known(UUID.randomUUID(), TEST_URL);
+				testingLeaderRetrievalDriver.onUpdate(newLeader);
+				testingListener.waitForNewLeader(timeout);
+				assertThat(testingListener.getLeaderSessionID(), is(newLeader.getLeaderSessionID()));
+				assertThat(testingListener.getAddress(), is(newLeader.getLeaderAddress()));
+			});
+		}};
+	}
+
+	@Test
+	public void testNotifyLeaderAddressEmpty() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation newLeader = LeaderInformation.known(UUID.randomUUID(), TEST_URL);
+				testingLeaderRetrievalDriver.onUpdate(newLeader);
+				testingListener.waitForNewLeader(timeout);
+
+				testingLeaderRetrievalDriver.onUpdate(LeaderInformation.empty());
+				testingListener.waitForEmptyLeaderInformation(timeout);
+				assertThat(testingListener.getLeaderSessionID(), is(nullValue()));
+				assertThat(testingListener.getAddress(), is(nullValue()));
+			});
+		}};
+	}
+
+	@Test
+	public void testErrorForwarding() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final Exception testException = new Exception("test Exeption");
+
+				testingLeaderRetrievalDriver.getFatalErrorHandler().onFatalError(testException);

Review comment:
       I would expose `onFatalError` directly on `testingLeaderRetrievalDriver` assuming that this is a testing implementation.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderRetrievalServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testNotifyLeaderAddress() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation newLeader = LeaderInformation.known(UUID.randomUUID(), TEST_URL);
+				testingLeaderRetrievalDriver.onUpdate(newLeader);
+				testingListener.waitForNewLeader(timeout);
+				assertThat(testingListener.getLeaderSessionID(), is(newLeader.getLeaderSessionID()));
+				assertThat(testingListener.getAddress(), is(newLeader.getLeaderAddress()));
+			});
+		}};
+	}
+
+	@Test
+	public void testNotifyLeaderAddressEmpty() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation newLeader = LeaderInformation.known(UUID.randomUUID(), TEST_URL);
+				testingLeaderRetrievalDriver.onUpdate(newLeader);
+				testingListener.waitForNewLeader(timeout);
+
+				testingLeaderRetrievalDriver.onUpdate(LeaderInformation.empty());
+				testingListener.waitForEmptyLeaderInformation(timeout);
+				assertThat(testingListener.getLeaderSessionID(), is(nullValue()));
+				assertThat(testingListener.getAddress(), is(nullValue()));
+			});
+		}};
+	}
+
+	@Test
+	public void testErrorForwarding() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final Exception testException = new Exception("test Exeption");
+
+				testingLeaderRetrievalDriver.getFatalErrorHandler().onFatalError(testException);
+
+				assertThat(testingListener.getError().getMessage(), containsString(testException.getMessage()));
+			});
+		}};
+	}
+
+	@Test
+	public void testErrorHappenAfterStop() throws Exception {

Review comment:
       nit: `testErrorHappensAfterStop` or even better, `testErrorIsIgnoredAfterBeingStopped`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				// grant leadership
+				testingLeaderElectionDriver.isLeader();
+
+				testingContender.waitForLeader(timeout);
+				assertThat(testingContender.getDescription(), is(TEST_URL));
+				assertThat(testingContender.getLeaderSessionID(), is(leaderElectionService.getLeaderSessionID()));
+				// Check the external storage
+				assertThat(
+					testingLeaderElectionDriver.getLeaderInformation(),
+					is(LeaderInformation.known(leaderElectionService.getLeaderSessionID(), TEST_URL)));
+
+				// revoke leadership
+				testingLeaderElectionDriver.notLeader();
+				testingContender.waitForRevokeLeader(timeout);
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+				assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue()));
+				// External storage should be cleared
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				// Leader information changed on external storage. It should be corrected.
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+
+				testingLeaderElectionDriver.leaderInformationChanged(
+					LeaderInformation.known(UUID.randomUUID(), "faulty-address"));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+			});
+		}};
+	}
+
+	@Test
+	public void testHasLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(true));
+				assertThat(leaderElectionService.hasLeadership(UUID.randomUUID()), is(false));
+
+				leaderElectionService.stop();
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(false));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedWhenNoConfirmedSessionID() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation faultyLeader = LeaderInformation.known(UUID.randomUUID(), "faulty-address");
+				testingLeaderElectionDriver.leaderInformationChanged(faultyLeader);
+				// External storage should keep the wrong value.
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(faultyLeader));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnGrantLeadershipHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.isLeader();
+				// leader contender is not granted leadership
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnLeaderInformationChangeHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+
+				// External storage should not be corrected
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnRevokeLeadershipHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID oldSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(testingContender.getLeaderSessionID(), is(oldSessionId));
+
+				leaderElectionService.stop();
+
+				testingLeaderElectionDriver.notLeader();
+				// leader contender is not revoked leadership
+				assertThat(testingContender.getLeaderSessionID(), is(oldSessionId));
+			});
+		}};
+	}
+
+	@Test
+	public void testOldConfirmLeaderInformation() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+
+				// Old confirm call should not be issued.
+				leaderElectionService.confirmLeadership(UUID.randomUUID(), TEST_URL);
+				assertThat(leaderElectionService.getLeaderSessionID(), is(currentLeaderSessionId));
+			});
+		}};
+	}
+
+	@Test
+	public void testErrorForwarding() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final Exception testException = new Exception("test leader Exeption");
+
+				testingLeaderElectionDriver.getFatalErrorHandler().onFatalError(testException);
+
+				testingContender.waitForError(timeout);
+				assertThat(testingContender.getError(), is(notNullValue()));
+				assertThat(testingContender.getError().getMessage(), containsString(testException.getMessage()));

Review comment:
       maybe use `assertThat(testingContender.getError(), FlinkMatchers.containsCause(testException))`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				// grant leadership
+				testingLeaderElectionDriver.isLeader();
+
+				testingContender.waitForLeader(timeout);
+				assertThat(testingContender.getDescription(), is(TEST_URL));
+				assertThat(testingContender.getLeaderSessionID(), is(leaderElectionService.getLeaderSessionID()));
+				// Check the external storage
+				assertThat(
+					testingLeaderElectionDriver.getLeaderInformation(),
+					is(LeaderInformation.known(leaderElectionService.getLeaderSessionID(), TEST_URL)));
+
+				// revoke leadership
+				testingLeaderElectionDriver.notLeader();
+				testingContender.waitForRevokeLeader(timeout);
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+				assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue()));
+				// External storage should be cleared
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				// Leader information changed on external storage. It should be corrected.
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+
+				testingLeaderElectionDriver.leaderInformationChanged(
+					LeaderInformation.known(UUID.randomUUID(), "faulty-address"));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+			});
+		}};
+	}
+
+	@Test
+	public void testHasLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(true));
+				assertThat(leaderElectionService.hasLeadership(UUID.randomUUID()), is(false));
+
+				leaderElectionService.stop();
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(false));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedWhenNoConfirmedSessionID() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation faultyLeader = LeaderInformation.known(UUID.randomUUID(), "faulty-address");
+				testingLeaderElectionDriver.leaderInformationChanged(faultyLeader);
+				// External storage should keep the wrong value.
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(faultyLeader));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnGrantLeadershipHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.isLeader();
+				// leader contender is not granted leadership
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnLeaderInformationChangeHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+
+				// External storage should not be corrected
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnRevokeLeadershipHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID oldSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(testingContender.getLeaderSessionID(), is(oldSessionId));
+
+				leaderElectionService.stop();
+
+				testingLeaderElectionDriver.notLeader();
+				// leader contender is not revoked leadership
+				assertThat(testingContender.getLeaderSessionID(), is(oldSessionId));
+			});
+		}};
+	}
+
+	@Test
+	public void testOldConfirmLeaderInformation() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+
+				// Old confirm call should not be issued.
+				leaderElectionService.confirmLeadership(UUID.randomUUID(), TEST_URL);
+				assertThat(leaderElectionService.getLeaderSessionID(), is(currentLeaderSessionId));
+			});
+		}};
+	}
+
+	@Test
+	public void testErrorForwarding() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final Exception testException = new Exception("test leader Exeption");
+
+				testingLeaderElectionDriver.getFatalErrorHandler().onFatalError(testException);

Review comment:
       I guess that `testingLeaderElectionDriver` is a testing implementation. Hence, we could directly expose `onFatalError` there. Otherwise we are exposing internal structure of this class which violates the law of demeter.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderRetrievalServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testNotifyLeaderAddress() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation newLeader = LeaderInformation.known(UUID.randomUUID(), TEST_URL);
+				testingLeaderRetrievalDriver.onUpdate(newLeader);
+				testingListener.waitForNewLeader(timeout);
+				assertThat(testingListener.getLeaderSessionID(), is(newLeader.getLeaderSessionID()));
+				assertThat(testingListener.getAddress(), is(newLeader.getLeaderAddress()));
+			});
+		}};
+	}
+
+	@Test
+	public void testNotifyLeaderAddressEmpty() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation newLeader = LeaderInformation.known(UUID.randomUUID(), TEST_URL);
+				testingLeaderRetrievalDriver.onUpdate(newLeader);
+				testingListener.waitForNewLeader(timeout);
+
+				testingLeaderRetrievalDriver.onUpdate(LeaderInformation.empty());
+				testingListener.waitForEmptyLeaderInformation(timeout);
+				assertThat(testingListener.getLeaderSessionID(), is(nullValue()));
+				assertThat(testingListener.getAddress(), is(nullValue()));
+			});
+		}};
+	}
+
+	@Test
+	public void testErrorForwarding() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final Exception testException = new Exception("test Exeption");
+
+				testingLeaderRetrievalDriver.getFatalErrorHandler().onFatalError(testException);
+
+				assertThat(testingListener.getError().getMessage(), containsString(testException.getMessage()));

Review comment:
       `assertThat(testingListener.getError(), FlinkMatchers.containsCause(testException))`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
##########
@@ -520,6 +519,47 @@ public void testEphemeralZooKeeperNodes() throws Exception {
 		}
 	}
 
+	@Test
+	public void testNotLeader() throws Exception {
+
+		final TestingLeaderElectionEventHandler electionEventHandler =
+			new TestingLeaderElectionEventHandler(TEST_LEADER);
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+		LeaderElectionDriver leaderElectionDriver = null;
+		LeaderRetrievalDriver leaderRetrievalDriver = null;
+		try {

Review comment:
       nit: line breaks before `try {` help the readability.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+
+		client.getConnectionStateListenable().removeListener(listener);
+
+		Exception exception = null;
+
+		try {
+			cache.close();
+		} catch (Exception e) {
+			exception = e;
+		}
+
+		try {
+			leaderLatch.close();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		assert(running);
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		leaderElectionEventHandler.onGrantLeadership();
+	}
+
+	@Override
+	public void notLeader() {
+		leaderElectionEventHandler.onRevokeLeadership();
+	}
+
+	@Override
+	public void nodeChanged() throws Exception {
+		if (leaderLatch.hasLeadership()) {
+			ChildData childData = cache.getCurrentData();
+			if (childData != null) {
+				final byte[] data = childData.getData();
+				if (data != null && data.length > 0) {
+					final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+					final ObjectInputStream ois = new ObjectInputStream(bais);
+
+					final String leaderAddress = ois.readUTF();
+					final UUID leaderSessionID = (UUID) ois.readObject();
+
+					leaderElectionEventHandler.onLeaderInformationChange(
+						LeaderInformation.known(leaderSessionID, leaderAddress));
+					return;
+				}
+			}
+			leaderElectionEventHandler.onLeaderInformationChange(LeaderInformation.empty());
+		}
+	}
+
+	/**
+	 * Writes the current leader's address as well the given leader session ID to ZooKeeper.
+	 */
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		assert(running);
+		// this method does not have to be synchronized because the curator framework client
+		// is thread-safe. We do not write the empty data to ZooKeeper here. Because check-leadership-and-update
+		// is not a transactional operation. We may wrongly clear the data written by new leader.
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Write leader information: {}.", leaderInformation);
+		}
+		if (leaderInformation.equals(LeaderInformation.empty())) {
+			return;
+		}
+		try {

Review comment:
       nit: line breaks between blocks can help to readability of the code.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientITCase.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.kubeclient;
+
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client.
+ */
+public class Fabric8FlinkKubeClientITCase {

Review comment:
       `extends TestLogger` is missing

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientITCase.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.kubeclient;
+
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client.
+ */
+public class Fabric8FlinkKubeClientITCase {
+
+	@ClassRule
+	public static KubernetesResource kubernetesResource = new KubernetesResource();
+
+	private static final String TEST_CONFIG_MAP_NAME = "test-config-map";
+
+	private static final long TIMEOUT = 120L * 1000L;
+
+	private static final Map<String, String> data = new HashMap<String, String>() {
+		{
+			put("key1", "0");
+			put("key2", "0");
+			put("key3", "0");
+		}
+	};
+
+	private FlinkKubeClient flinkKubeClient;
+
+	@Before
+	public void setup() throws Exception {
+		flinkKubeClient = kubernetesResource.getFlinkKubeClient();
+		flinkKubeClient.createConfigMap(new KubernetesConfigMap(
+			new ConfigMapBuilder()
+				.withNewMetadata()
+				.withName(TEST_CONFIG_MAP_NAME)
+				.endMetadata()
+				.withData(data)
+				.build())).get();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		flinkKubeClient.deleteConfigMap(TEST_CONFIG_MAP_NAME).get();
+	}
+
+	/**
+	 * {@link org.apache.flink.kubernetes.kubeclient.FlinkKubeClient#checkAndUpdateConfigMap} is a transactional
+	 * operation, we should definitely guarantee that the concurrent modification could work.
+	 */
+	@Test
+	public void testCheckAndUpdateConfigMapConcurrently() throws Exception {
+		// Start multiple instances to update ConfigMap concurrently
+		final List<CompletableFuture<Void>> futures = new ArrayList<>();
+		final int target = 10;
+		final int updateIntervalMs = 100;
+		for (String key : data.keySet()) {
+			futures.add(FutureUtils.runAfterwardsAsync(FutureUtils.completedVoidFuture(), () -> {

Review comment:
       Why are you using `FutureUtils.runAfterwardsAsync(FutureUtils.completedVoidFuture(), () -> {}, exec)` and not `CompletableFuture.runAsync(() -> {}, exec)`?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ * run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ *
+ * <p>{@link LeaderElector#run()} is responsible for creating the leader ConfigMap and continuously update the
+ * annotation. The annotation key is {@link #LEADER_ANNOTATION_KEY} and the value is in the following json format.
+ * metadata:
+ *   annotations:
+ *     control-plane.alpha.kubernetes.io/leader: '{"holderIdentity":"623e39fb-70c3-44f1-811f-561ec4a28d75","leaseDuration":15.000000000,"acquireTime":"2020-10-20T04:06:31.431000Z","renewTime":"2020-10-22T08:51:36.843000Z","leaderTransitions":37981}'
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {

Review comment:
       Why are we extending from `LeaderElector`? It looks that we only override the `run` method which has now a different semantic. Wouldn't it be enough to let `KubernetesLeaderElector` simply use a `LeaderElector` instance?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final FlinkKubeClient kubeClient;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		checkNotNull(leaderConfig, "Leader election configuration");
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElector.run();
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}.", this);
+		leaderElector.stop();
+		kubernetesWatch.close();
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		assert(running);
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// Get the updated ConfigMap with new leader information
+						if (confirmedLeaderAddress == null) {
+							configMap.getData().remove(LEADER_ADDRESS_KEY);
+						} else {
+							configMap.getData().put(LEADER_ADDRESS_KEY, confirmedLeaderAddress);
+						}
+						if (confirmedLeaderSessionID == null) {
+							configMap.getData().remove(LEADER_SESSION_ID_KEY);
+						} else {
+							configMap.getData().put(LEADER_SESSION_ID_KEY, confirmedLeaderSessionID.toString());
+						}
+						configMap.getLabels().putAll(configMapLabels);
+						return Optional.of(configMap);
+					}
+					return Optional.empty();
+				}).get();
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(
+					"Successfully wrote leader information: Leader={}, session ID={}.",
+					confirmedLeaderAddress,
+					confirmedLeaderSessionID);
+			}
+		} catch (Exception e) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not write leader information since ConfigMap " + configMapName
+					+ " does not exist.", e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		assert(running);
+		final Optional<KubernetesConfigMap> configMapOpt = kubeClient.getConfigMap(configMapName);
+		if (configMapOpt.isPresent()) {
+			return KubernetesLeaderElector.hasLeadership(configMapOpt.get(), lockIdentity);
+		} else {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("ConfigMap " + configMapName + " does not exist.", null));
+			return false;
+		}
+	}
+
+	private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+		@Override
+		public void isLeader() {
+			leaderElectionEventHandler.onGrantLeadership();
+		}
+
+		@Override
+		public void notLeader() {
+			leaderElectionEventHandler.onRevokeLeadership();
+			// Continue to contend the leader
+			leaderElector.run();
+		}
+	}
+
+	private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> {
+		@Override
+		public void onAdded(List<KubernetesConfigMap> configMaps) {
+			// noop
+		}
+
+		@Override
+		public void onModified(List<KubernetesConfigMap> configMaps) {
+			// We should only receive events for the watched ConfigMap
+			final KubernetesConfigMap configMap = checkConfigMaps(configMaps, configMapName);
+
+			if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+				leaderElectionEventHandler.onLeaderInformationChange(getLeaderInformationFromConfigMap(configMap));

Review comment:
       Are we sure that the leader information has been changed? What if we added a job to the table of running jobs or a completed checkpoint? This would also trigger this call unless we use different config maps for the different information.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/TestingLeaderCallbackHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.kubeclient.resources;
+
+import java.util.concurrent.BlockingQueue;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Testing implementation for {@link KubernetesLeaderElector.LeaderCallbackHandler}.
+ */
+public class TestingLeaderCallbackHandler extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+	private final BlockingQueue<String> leaderStore;
+	private final String lockIdentity;
+	private boolean isLeader;
+
+	public TestingLeaderCallbackHandler(BlockingQueue<String> leaderStore, String lockIdentity) {
+		this.leaderStore = leaderStore;
+		this.lockIdentity = lockIdentity;
+	}
+
+	@Override
+	public void isLeader() {
+		isLeader = true;
+		leaderStore.poll();
+		leaderStore.offer(lockIdentity);
+		assertThat(leaderStore.size(), is(1));

Review comment:
       Why is this necessary? It looks as if we only store the latest leader in `leaderStore`, right? Wouldn't it make sense to encapsulate this logic in `TestingLeaderCallbackHandler.waitForNewLeader` or so? Providing a `leaderStore` to this class and then having the contract that only the last leader is stored there seems not very intuitive. Moreover, what if the test thread polls the latest leader before `assertThat(leaderStore.size(), is(1))` runs?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderRetrievalListener} and
+ * {@link org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler}.
+ */
+public class TestingRetrievalBase {
+
+	private final BlockingQueue<LeaderInformation> leaderQueue = new LinkedBlockingQueue<>();
+
+	private String address;
+	private String oldAddress;
+	private UUID leaderSessionID;
+	private Exception exception;
+
+	public String waitForNewLeader(long timeout) throws Exception {
+		final LeaderInformation leader = leaderQueue.poll(timeout, TimeUnit.MILLISECONDS);

Review comment:
       before polling, we should check `exception` and throw it if it is not null.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.KubernetesResource;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderretrieval.TestingLeaderRetrievalEventHandler;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_CONFIGMAP_NAME;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_INFORMATION;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for the {@link KubernetesLeaderElectionDriver} and {@link KubernetesLeaderRetrievalDriver}. We expect the
+ * {@link KubernetesLeaderElectionDriver} could become the leader and {@link KubernetesLeaderRetrievalDriver} could
+ * retrieve the leader address from Kubernetes.
+ */
+public class KubernetesLeaderElectionAndRetrievalITCase {

Review comment:
       `extends TestLogger` is missing

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				// grant leadership
+				testingLeaderElectionDriver.isLeader();
+
+				testingContender.waitForLeader(timeout);
+				assertThat(testingContender.getDescription(), is(TEST_URL));
+				assertThat(testingContender.getLeaderSessionID(), is(leaderElectionService.getLeaderSessionID()));
+				// Check the external storage
+				assertThat(
+					testingLeaderElectionDriver.getLeaderInformation(),
+					is(LeaderInformation.known(leaderElectionService.getLeaderSessionID(), TEST_URL)));
+
+				// revoke leadership
+				testingLeaderElectionDriver.notLeader();
+				testingContender.waitForRevokeLeader(timeout);
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+				assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue()));
+				// External storage should be cleared
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				// Leader information changed on external storage. It should be corrected.
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+
+				testingLeaderElectionDriver.leaderInformationChanged(
+					LeaderInformation.known(UUID.randomUUID(), "faulty-address"));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));

Review comment:
       Same here. We might be able to write `assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.known(x, y)))`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory);
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		synchronized (lock) {
+			leaderContender = contender;
+			leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver(
+				this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription());
+			LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping DefaultLeaderElectionService.");
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.close();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+							"LeaderElectionService has already been stopped.", leaderSessionID);
+					}
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+							"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+					}
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			if (running) {
+				return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("hasLeadership is called after the service is stopped, returning false.");
+				}
+				return false;
+			}
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+		leaderElectionDriver.writeLeaderInformation(
+			LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress));
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+
+				leaderContender.grantLeadership(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the grant leadership notification since the {} has " +
+						"already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onRevokeLeadership() {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+				}
+
+				issuedLeaderSessionID = null;
+				clearConfirmedLeaderInformation();
+
+				leaderContender.revokeLeadership();
+
+				// Clear the old leader information on the external storage
+				leaderElectionDriver.writeLeaderInformation(LeaderInformation.empty());
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the revoke leadership notification since the {} " +
+						"has already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onLeaderInformationChange(LeaderInformation leaderInformation) {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Leader node changed while {} is the leader with session ID {}.",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID);
+				}
+				if (confirmedLeaderSessionID != null) {
+					final UUID leaderSessionID = leaderInformation.getLeaderSessionID();
+					final String leaderAddress = leaderInformation.getLeaderAddress();
+					final LeaderInformation confirmedLeaderInfo = LeaderInformation.known(
+						confirmedLeaderSessionID, confirmedLeaderAddress);
+					if (leaderSessionID == null && leaderAddress == null) {
+						if (LOG.isDebugEnabled()) {
+							LOG.debug("Writing leader information by {} since the external storage is empty.",
+								leaderContender.getDescription());
+						}
+						leaderElectionDriver.writeLeaderInformation(confirmedLeaderInfo);
+					} else if (!Objects.equals(leaderAddress, confirmedLeaderAddress) ||
+						!Objects.equals(leaderSessionID, confirmedLeaderSessionID)) {

Review comment:
       `!leaderInformation.equals(confirmedLeaderInfo)`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				// grant leadership
+				testingLeaderElectionDriver.isLeader();
+
+				testingContender.waitForLeader(timeout);
+				assertThat(testingContender.getDescription(), is(TEST_URL));
+				assertThat(testingContender.getLeaderSessionID(), is(leaderElectionService.getLeaderSessionID()));
+				// Check the external storage
+				assertThat(
+					testingLeaderElectionDriver.getLeaderInformation(),
+					is(LeaderInformation.known(leaderElectionService.getLeaderSessionID(), TEST_URL)));
+
+				// revoke leadership
+				testingLeaderElectionDriver.notLeader();
+				testingContender.waitForRevokeLeader(timeout);
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+				assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue()));
+				// External storage should be cleared
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				// Leader information changed on external storage. It should be corrected.
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+
+				testingLeaderElectionDriver.leaderInformationChanged(
+					LeaderInformation.known(UUID.randomUUID(), "faulty-address"));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+			});
+		}};
+	}
+
+	@Test
+	public void testHasLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(true));
+				assertThat(leaderElectionService.hasLeadership(UUID.randomUUID()), is(false));
+
+				leaderElectionService.stop();
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(false));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedWhenNoConfirmedSessionID() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation faultyLeader = LeaderInformation.known(UUID.randomUUID(), "faulty-address");
+				testingLeaderElectionDriver.leaderInformationChanged(faultyLeader);
+				// External storage should keep the wrong value.
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(faultyLeader));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnGrantLeadershipHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.isLeader();
+				// leader contender is not granted leadership
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnLeaderInformationChangeHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+
+				// External storage should not be corrected
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnRevokeLeadershipHappenAfterStop() throws Exception {

Review comment:
       Nit: `testOnRevokeLeadershipHappensAfterStop`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory);
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		synchronized (lock) {
+			leaderContender = contender;
+			leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver(
+				this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription());
+			LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping DefaultLeaderElectionService.");
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.close();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+							"LeaderElectionService has already been stopped.", leaderSessionID);
+					}
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+							"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+					}
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			if (running) {
+				return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("hasLeadership is called after the service is stopped, returning false.");
+				}
+				return false;
+			}
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+		leaderElectionDriver.writeLeaderInformation(
+			LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress));
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+
+				leaderContender.grantLeadership(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the grant leadership notification since the {} has " +
+						"already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onRevokeLeadership() {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+				}
+
+				issuedLeaderSessionID = null;
+				clearConfirmedLeaderInformation();
+
+				leaderContender.revokeLeadership();
+
+				// Clear the old leader information on the external storage
+				leaderElectionDriver.writeLeaderInformation(LeaderInformation.empty());
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the revoke leadership notification since the {} " +
+						"has already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onLeaderInformationChange(LeaderInformation leaderInformation) {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Leader node changed while {} is the leader with session ID {}.",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID);
+				}
+				if (confirmedLeaderSessionID != null) {
+					final UUID leaderSessionID = leaderInformation.getLeaderSessionID();
+					final String leaderAddress = leaderInformation.getLeaderAddress();
+					final LeaderInformation confirmedLeaderInfo = LeaderInformation.known(
+						confirmedLeaderSessionID, confirmedLeaderAddress);
+					if (leaderSessionID == null && leaderAddress == null) {
+						if (LOG.isDebugEnabled()) {
+							LOG.debug("Writing leader information by {} since the external storage is empty.",
+								leaderContender.getDescription());
+						}
+						leaderElectionDriver.writeLeaderInformation(confirmedLeaderInfo);
+					} else if (!Objects.equals(leaderAddress, confirmedLeaderAddress) ||
+						!Objects.equals(leaderSessionID, confirmedLeaderSessionID)) {
+						// the data field does not correspond to the expected leader information
+						if (LOG.isDebugEnabled()) {
+							LOG.debug("Correcting leader information by {}.", leaderContender.getDescription());
+						}
+						leaderElectionDriver.writeLeaderInformation(confirmedLeaderInfo);
+					}
+				}
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring change notification since the {} has " +
+						"already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	private class LeaderElectionFatalErrorHandler implements FatalErrorHandler {
+
+		@Override
+		public void onFatalError(Throwable throwable) {
+			synchronized (lock) {
+				if (running) {
+					if (throwable instanceof LeaderElectionException) {
+						leaderContender.handleError((LeaderElectionException) throwable);

Review comment:
       I would not call the `leaderContender` under the `lock` because of the risk of introducing potential deadlocks.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
##########
@@ -520,6 +519,47 @@ public void testEphemeralZooKeeperNodes() throws Exception {
 		}
 	}
 
+	@Test
+	public void testNotLeader() throws Exception {
+
+		final TestingLeaderElectionEventHandler electionEventHandler =
+			new TestingLeaderElectionEventHandler(TEST_LEADER);
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+		LeaderElectionDriver leaderElectionDriver = null;
+		LeaderRetrievalDriver leaderRetrievalDriver = null;
+		try {
+
+			leaderElectionDriver =
+				ZooKeeperUtils.createLeaderElectionDriverFactory(client, configuration)
+					.createLeaderElectionDriver(electionEventHandler, electionEventHandler::handleError, TEST_URL);
+			electionEventHandler.init(leaderElectionDriver);
+
+			electionEventHandler.waitForLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(TEST_LEADER));
+
+			// Leader is revoked
+			((ZooKeeperLeaderElectionDriver) leaderElectionDriver).notLeader();
+			electionEventHandler.waitForRevokeLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LeaderInformation.empty()));
+			// The data on ZooKeeper it not be cleared

Review comment:
       This seems to be a different behaviour than the K8s implementation. Why is this necessary?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				// grant leadership
+				testingLeaderElectionDriver.isLeader();
+
+				testingContender.waitForLeader(timeout);
+				assertThat(testingContender.getDescription(), is(TEST_URL));
+				assertThat(testingContender.getLeaderSessionID(), is(leaderElectionService.getLeaderSessionID()));
+				// Check the external storage
+				assertThat(
+					testingLeaderElectionDriver.getLeaderInformation(),
+					is(LeaderInformation.known(leaderElectionService.getLeaderSessionID(), TEST_URL)));
+
+				// revoke leadership
+				testingLeaderElectionDriver.notLeader();
+				testingContender.waitForRevokeLeader(timeout);
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+				assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue()));
+				// External storage should be cleared
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				// Leader information changed on external storage. It should be corrected.
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+
+				testingLeaderElectionDriver.leaderInformationChanged(
+					LeaderInformation.known(UUID.randomUUID(), "faulty-address"));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+			});
+		}};
+	}
+
+	@Test
+	public void testHasLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(true));
+				assertThat(leaderElectionService.hasLeadership(UUID.randomUUID()), is(false));
+
+				leaderElectionService.stop();
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(false));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedWhenNoConfirmedSessionID() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation faultyLeader = LeaderInformation.known(UUID.randomUUID(), "faulty-address");
+				testingLeaderElectionDriver.leaderInformationChanged(faultyLeader);
+				// External storage should keep the wrong value.
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(faultyLeader));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnGrantLeadershipHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.isLeader();
+				// leader contender is not granted leadership
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnLeaderInformationChangeHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+
+				// External storage should not be corrected
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnRevokeLeadershipHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID oldSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(testingContender.getLeaderSessionID(), is(oldSessionId));
+
+				leaderElectionService.stop();
+
+				testingLeaderElectionDriver.notLeader();
+				// leader contender is not revoked leadership
+				assertThat(testingContender.getLeaderSessionID(), is(oldSessionId));
+			});
+		}};
+	}
+
+	@Test
+	public void testOldConfirmLeaderInformation() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+
+				// Old confirm call should not be issued.

Review comment:
       maybe better: old confirm call should be ignored

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				// grant leadership
+				testingLeaderElectionDriver.isLeader();
+
+				testingContender.waitForLeader(timeout);
+				assertThat(testingContender.getDescription(), is(TEST_URL));
+				assertThat(testingContender.getLeaderSessionID(), is(leaderElectionService.getLeaderSessionID()));
+				// Check the external storage
+				assertThat(
+					testingLeaderElectionDriver.getLeaderInformation(),
+					is(LeaderInformation.known(leaderElectionService.getLeaderSessionID(), TEST_URL)));
+
+				// revoke leadership
+				testingLeaderElectionDriver.notLeader();
+				testingContender.waitForRevokeLeader(timeout);
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+				assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue()));
+				// External storage should be cleared
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				// Leader information changed on external storage. It should be corrected.
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));

Review comment:
       I guess we could use `LeaderInformation.equals` here.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				// grant leadership
+				testingLeaderElectionDriver.isLeader();
+
+				testingContender.waitForLeader(timeout);
+				assertThat(testingContender.getDescription(), is(TEST_URL));
+				assertThat(testingContender.getLeaderSessionID(), is(leaderElectionService.getLeaderSessionID()));
+				// Check the external storage
+				assertThat(
+					testingLeaderElectionDriver.getLeaderInformation(),
+					is(LeaderInformation.known(leaderElectionService.getLeaderSessionID(), TEST_URL)));
+
+				// revoke leadership
+				testingLeaderElectionDriver.notLeader();
+				testingContender.waitForRevokeLeader(timeout);
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+				assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue()));
+				// External storage should be cleared
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				// Leader information changed on external storage. It should be corrected.
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+
+				testingLeaderElectionDriver.leaderInformationChanged(
+					LeaderInformation.known(UUID.randomUUID(), "faulty-address"));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+			});
+		}};
+	}
+
+	@Test
+	public void testHasLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(true));
+				assertThat(leaderElectionService.hasLeadership(UUID.randomUUID()), is(false));
+
+				leaderElectionService.stop();
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(false));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedWhenNoConfirmedSessionID() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation faultyLeader = LeaderInformation.known(UUID.randomUUID(), "faulty-address");
+				testingLeaderElectionDriver.leaderInformationChanged(faultyLeader);
+				// External storage should keep the wrong value.
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(faultyLeader));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnGrantLeadershipHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.isLeader();
+				// leader contender is not granted leadership
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnLeaderInformationChangeHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				leaderElectionService.stop();
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+
+				// External storage should not be corrected
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnRevokeLeadershipHappenAfterStop() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID oldSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(testingContender.getLeaderSessionID(), is(oldSessionId));
+
+				leaderElectionService.stop();
+
+				testingLeaderElectionDriver.notLeader();
+				// leader contender is not revoked leadership
+				assertThat(testingContender.getLeaderSessionID(), is(oldSessionId));
+			});
+		}};
+	}
+
+	@Test
+	public void testOldConfirmLeaderInformation() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+
+				// Old confirm call should not be issued.
+				leaderElectionService.confirmLeadership(UUID.randomUUID(), TEST_URL);
+				assertThat(leaderElectionService.getLeaderSessionID(), is(currentLeaderSessionId));
+			});
+		}};
+	}
+
+	@Test
+	public void testErrorForwarding() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final Exception testException = new Exception("test leader Exeption");
+
+				testingLeaderElectionDriver.getFatalErrorHandler().onFatalError(testException);
+
+				testingContender.waitForError(timeout);
+				assertThat(testingContender.getError(), is(notNullValue()));
+				assertThat(testingContender.getError().getMessage(), containsString(testException.getMessage()));
+			});
+		}};
+	}
+
+	@Test
+	public void testErrorHappenAfterStop() throws Exception {

Review comment:
       nit: `testErrorHappenAfterStop`
   
   I think it is always a good practice to encode in the name of the test what the expected outcome is. So in your case, it would be `testErrorAfterStopIsIgnored`, for example.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+
+		client.getConnectionStateListenable().removeListener(listener);
+
+		Exception exception = null;
+
+		try {
+			cache.close();
+		} catch (Exception e) {
+			exception = e;
+		}
+
+		try {
+			leaderLatch.close();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		assert(running);
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		leaderElectionEventHandler.onGrantLeadership();
+	}
+
+	@Override
+	public void notLeader() {
+		leaderElectionEventHandler.onRevokeLeadership();
+	}
+
+	@Override
+	public void nodeChanged() throws Exception {
+		if (leaderLatch.hasLeadership()) {
+			ChildData childData = cache.getCurrentData();
+			if (childData != null) {
+				final byte[] data = childData.getData();
+				if (data != null && data.length > 0) {
+					final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+					final ObjectInputStream ois = new ObjectInputStream(bais);
+
+					final String leaderAddress = ois.readUTF();
+					final UUID leaderSessionID = (UUID) ois.readObject();
+
+					leaderElectionEventHandler.onLeaderInformationChange(
+						LeaderInformation.known(leaderSessionID, leaderAddress));
+					return;
+				}
+			}
+			leaderElectionEventHandler.onLeaderInformationChange(LeaderInformation.empty());
+		}
+	}
+
+	/**
+	 * Writes the current leader's address as well the given leader session ID to ZooKeeper.
+	 */
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		assert(running);
+		// this method does not have to be synchronized because the curator framework client
+		// is thread-safe. We do not write the empty data to ZooKeeper here. Because check-leadership-and-update
+		// is not a transactional operation. We may wrongly clear the data written by new leader.
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Write leader information: {}.", leaderInformation);
+		}
+		if (leaderInformation.equals(LeaderInformation.empty())) {

Review comment:
       We could have something like `LeaderInformation.isEmpty()`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+
+		client.getConnectionStateListenable().removeListener(listener);
+
+		Exception exception = null;
+
+		try {
+			cache.close();
+		} catch (Exception e) {
+			exception = e;
+		}
+
+		try {
+			leaderLatch.close();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		assert(running);
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		leaderElectionEventHandler.onGrantLeadership();
+	}
+
+	@Override
+	public void notLeader() {
+		leaderElectionEventHandler.onRevokeLeadership();
+	}
+
+	@Override
+	public void nodeChanged() throws Exception {
+		if (leaderLatch.hasLeadership()) {
+			ChildData childData = cache.getCurrentData();
+			if (childData != null) {
+				final byte[] data = childData.getData();
+				if (data != null && data.length > 0) {
+					final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+					final ObjectInputStream ois = new ObjectInputStream(bais);
+
+					final String leaderAddress = ois.readUTF();
+					final UUID leaderSessionID = (UUID) ois.readObject();
+
+					leaderElectionEventHandler.onLeaderInformationChange(
+						LeaderInformation.known(leaderSessionID, leaderAddress));
+					return;
+				}
+			}
+			leaderElectionEventHandler.onLeaderInformationChange(LeaderInformation.empty());
+		}
+	}
+
+	/**
+	 * Writes the current leader's address as well the given leader session ID to ZooKeeper.
+	 */
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		assert(running);
+		// this method does not have to be synchronized because the curator framework client
+		// is thread-safe. We do not write the empty data to ZooKeeper here. Because check-leadership-and-update
+		// is not a transactional operation. We may wrongly clear the data written by new leader.
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Write leader information: {}.", leaderInformation);
+		}
+		if (leaderInformation.equals(LeaderInformation.empty())) {
+			return;

Review comment:
       I believe that the ZK implementation behaves differently than the K8s implementation where we can delete leader information. Moreover, `DefaultLeaderElectionService` call `writeLeaderInformation(LeaderInformation.empty())` when the leadership is revoked. So what should be the normal behaviour?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
##########
@@ -520,6 +519,47 @@ public void testEphemeralZooKeeperNodes() throws Exception {
 		}
 	}
 
+	@Test
+	public void testNotLeader() throws Exception {
+
+		final TestingLeaderElectionEventHandler electionEventHandler =
+			new TestingLeaderElectionEventHandler(TEST_LEADER);
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+		LeaderElectionDriver leaderElectionDriver = null;
+		LeaderRetrievalDriver leaderRetrievalDriver = null;
+		try {
+
+			leaderElectionDriver =
+				ZooKeeperUtils.createLeaderElectionDriverFactory(client, configuration)
+					.createLeaderElectionDriver(electionEventHandler, electionEventHandler::handleError, TEST_URL);
+			electionEventHandler.init(leaderElectionDriver);
+
+			electionEventHandler.waitForLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(TEST_LEADER));
+
+			// Leader is revoked
+			((ZooKeeperLeaderElectionDriver) leaderElectionDriver).notLeader();

Review comment:
       Why not making `leaderElectionDriver` a variable of type `ZooKeeperLeaderElectionDriver`?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +271,131 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateConfigMap() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+		final Optional<KubernetesConfigMap> currentOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(currentOpt.isPresent(), is(true));
+		assertThat(currentOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	@Test
+	public void testCreateConfigMapAlreadyExisting() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+
+		mockCreateConfigMapAlreadyExisting(configMap.getInternalResource());
+		configMap.getData().put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_NEW_VALUE);
+		try {
+			this.flinkKubeClient.createConfigMap(configMap).get();
+			fail("Exception should be thrown.");

Review comment:
       Usually it is good to state why you fail (e.g. "Overwrite and already existing config map should fail with an exception.")

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientITCase.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.kubeclient;
+
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client.
+ */
+public class Fabric8FlinkKubeClientITCase {
+
+	@ClassRule
+	public static KubernetesResource kubernetesResource = new KubernetesResource();
+
+	private static final String TEST_CONFIG_MAP_NAME = "test-config-map";
+
+	private static final long TIMEOUT = 120L * 1000L;
+
+	private static final Map<String, String> data = new HashMap<String, String>() {
+		{
+			put("key1", "0");
+			put("key2", "0");
+			put("key3", "0");
+		}
+	};
+
+	private FlinkKubeClient flinkKubeClient;
+
+	@Before
+	public void setup() throws Exception {
+		flinkKubeClient = kubernetesResource.getFlinkKubeClient();
+		flinkKubeClient.createConfigMap(new KubernetesConfigMap(
+			new ConfigMapBuilder()
+				.withNewMetadata()
+				.withName(TEST_CONFIG_MAP_NAME)
+				.endMetadata()
+				.withData(data)
+				.build())).get();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		flinkKubeClient.deleteConfigMap(TEST_CONFIG_MAP_NAME).get();
+	}
+
+	/**
+	 * {@link org.apache.flink.kubernetes.kubeclient.FlinkKubeClient#checkAndUpdateConfigMap} is a transactional
+	 * operation, we should definitely guarantee that the concurrent modification could work.
+	 */
+	@Test
+	public void testCheckAndUpdateConfigMapConcurrently() throws Exception {
+		// Start multiple instances to update ConfigMap concurrently
+		final List<CompletableFuture<Void>> futures = new ArrayList<>();
+		final int target = 10;
+		final int updateIntervalMs = 100;
+		for (String key : data.keySet()) {
+			futures.add(FutureUtils.runAfterwardsAsync(FutureUtils.completedVoidFuture(), () -> {
+				for (int index = 0; index < target; index++) {
+					final boolean updated = flinkKubeClient.checkAndUpdateConfigMap(
+						TEST_CONFIG_MAP_NAME,
+						configMap -> {
+							final int newValue = Integer.valueOf(configMap.getData().get(key)) + 1;
+							configMap.getData().put(key, String.valueOf(newValue));
+							return Optional.of(configMap);
+						}).get();
+					assertThat(updated, is(true));
+					try {
+						// Simulate the update interval
+						Thread.sleep((long) (updateIntervalMs * Math.random()));
+					} catch (InterruptedException e) {
+						// noop
+					}
+				}
+			}, kubernetesResource.getExecutorService()));

Review comment:
       Why does the `KubernetesResource` expose the internal `ExecutorService` which is also used for the `flinkKubeClient`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
##########
@@ -118,43 +127,42 @@ public void after() throws IOException {
 	 */
 	@Test
 	public void testZooKeeperLeaderElectionRetrieval() throws Exception {
-		ZooKeeperLeaderElectionService leaderElectionService = null;
-		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
 
+		final TestingLeaderElectionEventHandler electionEventHandler =
+			new TestingLeaderElectionEventHandler(TEST_LEADER);
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+		LeaderElectionDriver leaderElectionDriver = null;
+		LeaderRetrievalDriver leaderRetrievalDriver = null;
 		try {
-			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(client, configuration);
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(client, configuration);
-
-			TestingContender contender = new TestingContender(TEST_URL, leaderElectionService);
-			TestingListener listener = new TestingListener();
-
-			leaderElectionService.start(contender);
-			leaderRetrievalService.start(listener);
-
-			contender.waitForLeader(timeout);
 
-			assertTrue(contender.isLeader());
-			assertEquals(leaderElectionService.getLeaderSessionID(), contender.getLeaderSessionID());
+			leaderElectionDriver =
+				ZooKeeperUtils.createLeaderElectionDriverFactory(client, configuration)
+					.createLeaderElectionDriver(electionEventHandler, electionEventHandler::handleError, TEST_URL);
+			electionEventHandler.init(leaderElectionDriver);

Review comment:
       Could this be factored out into a method which is reused?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java
##########
@@ -60,6 +62,11 @@
 
 	private CuratorFramework zooKeeperClient;
 
+	private final FatalErrorHandler fatalErrorHandler = throwable -> {
+		throw new UnsupportedOperationException(
+			"handleError(Exception) shouldn't have been called, but it was triggered anyway.", throwable);
+	};

Review comment:
       Maybe we could add a `DirectlyFailingFatalErrorHandler.INSTANCE` as a utility which can be used by other classes as well.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +271,131 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateConfigMap() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+		final Optional<KubernetesConfigMap> currentOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(currentOpt.isPresent(), is(true));
+		assertThat(currentOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	@Test
+	public void testCreateConfigMapAlreadyExisting() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+
+		mockCreateConfigMapAlreadyExisting(configMap.getInternalResource());
+		configMap.getData().put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_NEW_VALUE);
+		try {
+			this.flinkKubeClient.createConfigMap(configMap).get();
+			fail("Exception should be thrown.");
+		} catch (Exception ex) {
+			assertThat(ex.getMessage(),
+				containsString("Failed to create ConfigMap " + TESTING_CONFIG_MAP_NAME));

Review comment:
       The problem with this pattern is that in case of failure of the assertion we don't see the stack trace. Hence, in the future I would suggest to add a new matcher in `FlinkMatchers.containsMessage(message)` or so which checks the exception for the message and if not fails with also printing the stack trace of the exception.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ * run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ *
+ * <p>{@link LeaderElector#run()} is responsible for creating the leader ConfigMap and continuously update the
+ * annotation. The annotation key is {@link #LEADER_ANNOTATION_KEY} and the value is in the following json format.
+ * metadata:
+ *   annotations:
+ *     control-plane.alpha.kubernetes.io/leader: '{"holderIdentity":"623e39fb-70c3-44f1-811f-561ec4a28d75","leaseDuration":15.000000000,"acquireTime":"2020-10-20T04:06:31.431000Z","renewTime":"2020-10-22T08:51:36.843000Z","leaderTransitions":37981}'
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	@VisibleForTesting
+	public static final String LEADER_ANNOTATION_KEY = "control-plane.alpha.kubernetes.io/leader";
+
+	private final ExecutorService executorService = Executors.newSingleThreadExecutor(
+		new ExecutorThreadFactory("KubernetesLeaderElector-ExecutorService"));
+
+	public KubernetesLeaderElector(
+			NamespacedKubernetesClient kubernetesClient,
+			String namespace,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderCallbackHandler leaderCallbackHandler) {
+		super(kubernetesClient, new LeaderElectionConfigBuilder()
+			.withName(leaderConfig.getConfigMapName())
+			.withLeaseDuration(leaderConfig.getLeaseDuration())
+			.withLock(new ConfigMapLock(namespace, leaderConfig.getConfigMapName(), leaderConfig.getLockIdentity()))
+			.withRenewDeadline(leaderConfig.getRenewDeadline())
+			.withRetryPeriod(leaderConfig.getRetryPeriod())
+			.withLeaderCallbacks(new LeaderCallbacks(
+				leaderCallbackHandler::isLeader,
+				leaderCallbackHandler::notLeader,
+				newLeader -> LOG.info("New leader elected {} for {}.", newLeader, leaderConfig.getConfigMapName())
+			))
+			.build());
+		LOG.info("Create KubernetesLeaderElector {} with lock identity {}.",
+			leaderConfig.getConfigMapName(), leaderConfig.getLockIdentity());
+	}
+
+	@Override
+	public void run() {
+		CompletableFuture.runAsync(super::run, executorService);

Review comment:
       Why are you using `CompletableFuture.runAsync` here instead of `executorService.execute(super::run)`?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.DefaultKubeClientFactory;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubeClientFactory;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for the {@link KubernetesLeaderElector}. Start multiple leader contenders currently, one should elect
+ * successfully. And if current leader dies, a new one could take over.
+ */
+public class KubernetesLeaderElectorITCase {

Review comment:
       `extends TestLogger` is missing

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.leaderelection;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderContender} and
+ * {@link LeaderElectionEventHandler}.
+ */
+public class TestingLeaderBase {
+	// The queues will be offered by subclasses
+	protected final BlockingQueue<LeaderInformation> leaderQueue = new LinkedBlockingQueue<>();
+	protected final BlockingQueue<LeaderInformation> revokeQueue = new LinkedBlockingQueue<>();
+	protected final BlockingQueue<Throwable> errorQueue = new LinkedBlockingQueue<>();
+
+	private boolean isLeader = false;
+	private Throwable error;
+
+	public void waitForLeader(long timeout) throws Exception {
+		final LeaderInformation leader = leaderQueue.poll(timeout, TimeUnit.MILLISECONDS);
+		if (leader == null || leader.equals(LeaderInformation.empty())) {
+			throw new TimeoutException("Contender was not elected as the leader within " + timeout + "ms");
+		}
+		isLeader = true;
+	}
+
+	public void waitForRevokeLeader(long timeout) throws Exception {
+		final LeaderInformation revoke = revokeQueue.poll(timeout, TimeUnit.MILLISECONDS);
+		if (revoke == null || !revoke.equals(LeaderInformation.empty())) {

Review comment:
       Maybe we could unify `leaderQueue` and `revokeQueue` into a `leaderEventQueue` and then poll in `waitForLeader` from it until we see a non-empty leader information or the timeout occurs. In `waitForRevokeLeader` we would do the same until an empty leader information occurs.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final FlinkKubeClient kubeClient;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		checkNotNull(leaderConfig, "Leader election configuration");
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElector.run();

Review comment:
       Shouldn't we call `run` as the last thing? Otherwise we might see callbacks before `running` is set to `true`.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.runtime.leaderelection.LeaderInformation;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_KEY;
+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.hamcrest.Matchers.containsString;
+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 KubernetesLeaderElectionDriver}.
+ */
+public class KubernetesLeaderElectionDriverTest extends KubernetesHighAvailabilityTestBase {
+
+	@Test
+	public void testIsLeader() throws Exception {
+		new Context() {{
+			runTest(
+				() -> {
+					// Grant leadership
+					leaderCallbackGrantLeadership();
+					assertThat(electionEventHandler.isLeader(), is(true));
+					assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LEADER_INFORMATION));
+				});
+		}};
+	}
+
+	@Test
+	public void testNotLeader() throws Exception {
+		new Context() {{
+			runTest(
+				() -> {
+					leaderCallbackGrantLeadership();
+					// Revoke leadership
+					getLeaderCallback().notLeader();
+
+					electionEventHandler.waitForRevokeLeader(TIMEOUT);
+					assertThat(electionEventHandler.isLeader(), is(false));
+					assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LeaderInformation.empty()));
+					// The ConfigMap should also be cleared
+					assertThat(getLeaderConfigMap().getData().get(LEADER_ADDRESS_KEY), is(nullValue()));
+					assertThat(getLeaderConfigMap().getData().get(LEADER_SESSION_ID_KEY), is(nullValue()));

Review comment:
       How will this work if the `KubernetesLeaderElectionDriver` checks whether one is currently the leader before writing the leader information?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.containsString;
+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 {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest extends TestLogger {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				// grant leadership
+				testingLeaderElectionDriver.isLeader();
+
+				testingContender.waitForLeader(timeout);
+				assertThat(testingContender.getDescription(), is(TEST_URL));
+				assertThat(testingContender.getLeaderSessionID(), is(leaderElectionService.getLeaderSessionID()));
+				// Check the external storage
+				assertThat(
+					testingLeaderElectionDriver.getLeaderInformation(),
+					is(LeaderInformation.known(leaderElectionService.getLeaderSessionID(), TEST_URL)));
+
+				// revoke leadership
+				testingLeaderElectionDriver.notLeader();
+				testingContender.waitForRevokeLeader(timeout);
+				assertThat(testingContender.getLeaderSessionID(), is(nullValue()));
+				assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue()));
+				// External storage should be cleared
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(LeaderInformation.empty()));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+
+				// Leader information changed on external storage. It should be corrected.
+				testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+
+				testingLeaderElectionDriver.leaderInformationChanged(
+					LeaderInformation.known(UUID.randomUUID(), "faulty-address"));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+				assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderSessionID(),
+					is(leaderElectionService.getLeaderSessionID()));
+			});
+		}};
+	}
+
+	@Test
+	public void testHasLeadership() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				testingLeaderElectionDriver.isLeader();
+				final UUID currentLeaderSessionId = leaderElectionService.getLeaderSessionID();
+				assertThat(currentLeaderSessionId, is(notNullValue()));
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(true));
+				assertThat(leaderElectionService.hasLeadership(UUID.randomUUID()), is(false));
+
+				leaderElectionService.stop();
+				assertThat(leaderElectionService.hasLeadership(currentLeaderSessionId), is(false));
+			});
+		}};
+	}
+
+	@Test
+	public void testLeaderInformationChangedWhenNoConfirmedSessionID() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				final LeaderInformation faultyLeader = LeaderInformation.known(UUID.randomUUID(), "faulty-address");
+				testingLeaderElectionDriver.leaderInformationChanged(faultyLeader);
+				// External storage should keep the wrong value.
+				assertThat(testingLeaderElectionDriver.getLeaderInformation(), is(faultyLeader));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnGrantLeadershipHappenAfterStop() throws Exception {

Review comment:
       nit:  `testOnGrantLeadershipHappensAfterStop`

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ * run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ *
+ * <p>{@link LeaderElector#run()} is responsible for creating the leader ConfigMap and continuously update the
+ * annotation. The annotation key is {@link #LEADER_ANNOTATION_KEY} and the value is in the following json format.
+ * metadata:
+ *   annotations:
+ *     control-plane.alpha.kubernetes.io/leader: '{"holderIdentity":"623e39fb-70c3-44f1-811f-561ec4a28d75","leaseDuration":15.000000000,"acquireTime":"2020-10-20T04:06:31.431000Z","renewTime":"2020-10-22T08:51:36.843000Z","leaderTransitions":37981}'
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	@VisibleForTesting
+	public static final String LEADER_ANNOTATION_KEY = "control-plane.alpha.kubernetes.io/leader";
+
+	private final ExecutorService executorService = Executors.newSingleThreadExecutor(
+		new ExecutorThreadFactory("KubernetesLeaderElector-ExecutorService"));
+
+	public KubernetesLeaderElector(
+			NamespacedKubernetesClient kubernetesClient,
+			String namespace,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderCallbackHandler leaderCallbackHandler) {
+		super(kubernetesClient, new LeaderElectionConfigBuilder()
+			.withName(leaderConfig.getConfigMapName())
+			.withLeaseDuration(leaderConfig.getLeaseDuration())
+			.withLock(new ConfigMapLock(namespace, leaderConfig.getConfigMapName(), leaderConfig.getLockIdentity()))
+			.withRenewDeadline(leaderConfig.getRenewDeadline())
+			.withRetryPeriod(leaderConfig.getRetryPeriod())
+			.withLeaderCallbacks(new LeaderCallbacks(
+				leaderCallbackHandler::isLeader,
+				leaderCallbackHandler::notLeader,
+				newLeader -> LOG.info("New leader elected {} for {}.", newLeader, leaderConfig.getConfigMapName())
+			))
+			.build());
+		LOG.info("Create KubernetesLeaderElector {} with lock identity {}.",
+			leaderConfig.getConfigMapName(), leaderConfig.getLockIdentity());
+	}
+
+	@Override
+	public void run() {
+		CompletableFuture.runAsync(super::run, executorService);

Review comment:
       Can `super::run` throw an exception? I yes, then I think the uncaught exception handler of the `ExecutorThreadFactory` should handle it. Hence, I guess it should be fine.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.DefaultKubeClientFactory;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubeClientFactory;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for the {@link KubernetesLeaderElector}. Start multiple leader contenders currently, one should elect
+ * successfully. And if current leader dies, a new one could take over.
+ */
+public class KubernetesLeaderElectorITCase {
+
+	@ClassRule
+	public static KubernetesResource kubernetesResource = new KubernetesResource();
+
+	private static final long TIMEOUT = 120L * 1000L;
+
+	private final KubeClientFactory kubeClientFactory = new DefaultKubeClientFactory();
+
+	private static final String LEADER_CONFIGMAP_NAME_PREFIX = "leader-test-cluster";
+
+	@Test
+	public void testMultipleKubernetesLeaderElectors() throws Exception {
+		final Configuration configuration = kubernetesResource.getConfiguration();
+		final ExecutorService executorService = kubernetesResource.getExecutorService();
+
+		final BlockingQueue<String> leaderStore = new LinkedBlockingQueue<>();
+		final String leaderConfigMapName = LEADER_CONFIGMAP_NAME_PREFIX + System.currentTimeMillis();
+		final int leaderNum = 3;
+
+		final KubernetesLeaderElector[] leaderElectors = new KubernetesLeaderElector[leaderNum];
+		// We use different Kubernetes clients for different leader electors.
+		final FlinkKubeClient[] kubeClients = new FlinkKubeClient[leaderNum];
+		final TestingLeaderCallbackHandler[] leaderCallbackHandlers = new TestingLeaderCallbackHandler[leaderNum];
+
+		try {
+			for (int i = 0; i < leaderNum; i++) {
+				kubeClients[i] = kubeClientFactory.fromConfiguration(configuration, executorService);
+				leaderCallbackHandlers[i] = new TestingLeaderCallbackHandler(leaderStore, UUID.randomUUID().toString());
+				final KubernetesLeaderElectionConfiguration leaderConfig = new KubernetesLeaderElectionConfiguration(
+					leaderConfigMapName, leaderCallbackHandlers[i].getLockIdentity(), configuration);
+				leaderElectors[i] = kubeClients[i].createLeaderElector(leaderConfig, leaderCallbackHandlers[i]);
+
+				// Start the leader electors to contend the leader
+				leaderElectors[i].run();
+			}
+
+			// Wait for the first leader
+			final String firstLockIdentity = leaderStore.poll(TIMEOUT, TimeUnit.MILLISECONDS);
+
+			for (int i = 0; i < leaderNum; i++) {
+				if (leaderCallbackHandlers[i].getLockIdentity().equals(firstLockIdentity)) {
+					// Check the callback is called.
+					assertThat(leaderCallbackHandlers[i].hasLeadership(), is(true));
+					// Current leader died
+					leaderElectors[i].stop();

Review comment:
       When will `LeaderCallbackHandler.notLeader` be called?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory);
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		synchronized (lock) {
+			leaderContender = contender;
+			leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver(
+				this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription());
+			LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping DefaultLeaderElectionService.");
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.close();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+							"LeaderElectionService has already been stopped.", leaderSessionID);
+					}
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+							"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+					}
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			if (running) {
+				return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("hasLeadership is called after the service is stopped, returning false.");
+				}
+				return false;
+			}
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+		leaderElectionDriver.writeLeaderInformation(
+			LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress));
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+
+				leaderContender.grantLeadership(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the grant leadership notification since the {} has " +
+						"already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onRevokeLeadership() {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+				}
+
+				issuedLeaderSessionID = null;
+				clearConfirmedLeaderInformation();
+
+				leaderContender.revokeLeadership();
+
+				// Clear the old leader information on the external storage
+				leaderElectionDriver.writeLeaderInformation(LeaderInformation.empty());

Review comment:
       Will this have any effect? The `ZooKeeperLeaderServiceDriver` will ignore this call and the `KubernetesLeaderServiceDriver` won't do much either because it just got its leadership revoked and we check in the `checkAndUpdateConfig` method whether we still have the leadership.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+
+		client.getConnectionStateListenable().removeListener(listener);
+
+		Exception exception = null;
+
+		try {
+			cache.close();
+		} catch (Exception e) {
+			exception = e;
+		}
+
+		try {
+			leaderLatch.close();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		assert(running);
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		leaderElectionEventHandler.onGrantLeadership();
+	}
+
+	@Override
+	public void notLeader() {
+		leaderElectionEventHandler.onRevokeLeadership();
+	}
+
+	@Override
+	public void nodeChanged() throws Exception {
+		if (leaderLatch.hasLeadership()) {
+			ChildData childData = cache.getCurrentData();
+			if (childData != null) {
+				final byte[] data = childData.getData();
+				if (data != null && data.length > 0) {
+					final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+					final ObjectInputStream ois = new ObjectInputStream(bais);
+
+					final String leaderAddress = ois.readUTF();
+					final UUID leaderSessionID = (UUID) ois.readObject();
+
+					leaderElectionEventHandler.onLeaderInformationChange(
+						LeaderInformation.known(leaderSessionID, leaderAddress));
+					return;
+				}
+			}
+			leaderElectionEventHandler.onLeaderInformationChange(LeaderInformation.empty());
+		}
+	}
+
+	/**
+	 * Writes the current leader's address as well the given leader session ID to ZooKeeper.
+	 */
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		assert(running);
+		// this method does not have to be synchronized because the curator framework client
+		// is thread-safe. We do not write the empty data to ZooKeeper here. Because check-leadership-and-update
+		// is not a transactional operation. We may wrongly clear the data written by new leader.
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Write leader information: {}.", leaderInformation);
+		}
+		if (leaderInformation.equals(LeaderInformation.empty())) {
+			return;

Review comment:
       I guess we are stopping here because we cannot check that we currently have the leadership while updating the leader ship information. Maybe we should capture this specific behaviour in the JavaDocs of this method. Alternatively we could make it the responsibility of the user of this method to use this method correctly (e.g. not calling it with `LeaderInformation.empty()` if one does not have the leadership.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8720",
       "triggerID" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753",
       "triggerID" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884",
       "triggerID" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8956",
       "triggerID" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8988",
       "triggerID" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8956) 
   * e2795d9209fa2241cf0909737df9dca17defea4e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8988) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r517225129



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientITCase.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.kubeclient;
+
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client.
+ */
+public class Fabric8FlinkKubeClientITCase {
+
+	@ClassRule
+	public static KubernetesResource kubernetesResource = new KubernetesResource();
+
+	private static final String TEST_CONFIG_MAP_NAME = "test-config-map";
+
+	private static final long TIMEOUT = 120L * 1000L;
+
+	private static final Map<String, String> data = new HashMap<String, String>() {
+		{
+			put("key1", "0");
+			put("key2", "0");
+			put("key3", "0");
+		}
+	};
+
+	private FlinkKubeClient flinkKubeClient;
+
+	@Before
+	public void setup() throws Exception {
+		flinkKubeClient = kubernetesResource.getFlinkKubeClient();
+		flinkKubeClient.createConfigMap(new KubernetesConfigMap(
+			new ConfigMapBuilder()
+				.withNewMetadata()
+				.withName(TEST_CONFIG_MAP_NAME)
+				.endMetadata()
+				.withData(data)
+				.build())).get();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		flinkKubeClient.deleteConfigMap(TEST_CONFIG_MAP_NAME).get();
+	}
+
+	/**
+	 * {@link org.apache.flink.kubernetes.kubeclient.FlinkKubeClient#checkAndUpdateConfigMap} is a transactional
+	 * operation, we should definitely guarantee that the concurrent modification could work.
+	 */
+	@Test
+	public void testCheckAndUpdateConfigMapConcurrently() throws Exception {
+		// Start multiple instances to update ConfigMap concurrently
+		final List<CompletableFuture<Void>> futures = new ArrayList<>();
+		final int target = 10;
+		final int updateIntervalMs = 100;
+		for (String key : data.keySet()) {
+			futures.add(FutureUtils.runAfterwardsAsync(FutureUtils.completedVoidFuture(), () -> {

Review comment:
       I guess we could still fix this here via using `CompeltableFuture.runAsync()` and `join()` instead of `get()`. Very minor though.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507537261



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	protected volatile LeaderContender leaderContender;
+
+	private volatile UUID issuedLeaderSessionID;
+
+	protected volatile UUID confirmedLeaderSessionID;
+
+	protected volatile String confirmedLeaderAddress;
+
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+		}
+
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		internalStop();
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (logger.isDebugEnabled()) {
+			logger.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		if (checkLeaderLatch()) {

Review comment:
       I am afraid we could not completely get rid of `checkLeaderLatch` even we adjust the `if`s. `checkLeaderLatch` is used for checking whether the real leader elector(e.g. `LeaderLatch` for ZK and `KubernetesLeaderElector` for K8s) still have the leadership. Maybe we could rename the interface to `hasLeadership` without arguments. 
   
   Then the specific `LeaderElectorService` just need to implement the `hasLeadership()` via checking the real elector status.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516786488



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
##########
@@ -520,6 +519,47 @@ public void testEphemeralZooKeeperNodes() throws Exception {
 		}
 	}
 
+	@Test
+	public void testNotLeader() throws Exception {
+
+		final TestingLeaderElectionEventHandler electionEventHandler =
+			new TestingLeaderElectionEventHandler(TEST_LEADER);
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+		LeaderElectionDriver leaderElectionDriver = null;
+		LeaderRetrievalDriver leaderRetrievalDriver = null;
+		try {
+
+			leaderElectionDriver =
+				ZooKeeperUtils.createLeaderElectionDriverFactory(client, configuration)
+					.createLeaderElectionDriver(electionEventHandler, electionEventHandler::handleError, TEST_URL);
+			electionEventHandler.init(leaderElectionDriver);
+
+			electionEventHandler.waitForLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(TEST_LEADER));
+
+			// Leader is revoked
+			((ZooKeeperLeaderElectionDriver) leaderElectionDriver).notLeader();
+			electionEventHandler.waitForRevokeLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LeaderInformation.empty()));
+			// The data on ZooKeeper it not be cleared

Review comment:
       Sounds good.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-717116888


   Address @xintongsong 's comments and rebase master to resolve conflicts.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-713591200


   @tillrohrmann I have gone though all your comments and will update the PR soon. Just two quick points need to confirm with you.
   
   * Do you think we should not handle the externally deletion/update? For example, a Flink cluster with HA configured is running, some user delete/update the ConfigMap via `kubectl`. If it is yes, I will remove the operations in `KubernetesLeaderElectionService#Watcher`. And change some "ConfigMap not exists" behavior.
   * I am afraid it is hard to use a real K8s server in the UT because it is not very easy to start a `minikube`. I will try to add the unit tests for the contract testing now and leave the real cluster test in E2E test implementation. Does it make sense?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508993402



##########
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:
       Unlike Yarn MiniCluster, we are not very easy to start a minikube. So I prefer to add the unit tests for the contract testing first and leave the integration tests with E2E tests together. WDYT?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513904491



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}", this);
+
+			client.getUnhandledErrorListenable().removeListener(this);
+
+			client.getConnectionStateListenable().removeListener(listener);
+
+			Exception exception = null;
+
+			try {
+				cache.close();
+			} catch (Exception e) {
+				exception = e;
+			}
+
+			try {
+				leaderLatch.close();
+			} catch (Exception e) {
+				exception = ExceptionUtils.firstOrSuppressed(e, exception);
+			}
+
+			if (exception != null) {
+				throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		synchronized (lock) {
+			if (running) {
+				leaderElectionEventHandler.onGrantLeadership();
+			}
+		}

Review comment:
       I have noticed the deadlock problem and moved `leaderElectionDriver.close()` out of the outer lock. This will solve the deadlock problem now. But it is fragile and not very easy to understand. Maybe in the future we introduce some other interfaces, which will be called in the outer lock, then we still have the potential deadlock.
   
   Not execute the callback under the inner lock could completely avoid the deadlock. Just like you say, the implementation(e.g. `DefaultLeaderElection`) needs to filter out the spurious callbacks. This is easier to understand.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513218189



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalDriver.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link ZooKeeperLeaderElectionDriver}.
+ * {@link LeaderRetrievalService} implementation for Zookeeper. It retrieves the current leader which has
+ * been elected by the {@link ZooKeeperLeaderElectionDriver}.
+ * The leader address as well as the current leader session ID is retrieved from ZooKeeper.
+ */
+public class ZooKeeperLeaderRetrievalDriver implements LeaderRetrievalDriver, NodeCacheListener, UnhandledErrorListener {
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderRetrievalDriver.class);
+
+	/** Connection to the used ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe to watch changes of a specific ZooKeeper node. */
+	private final NodeCache cache;
+
+	private final String retrievalPath;
+
+	private final ConnectionStateListener connectionStateListener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderRetrievalEventHandler leaderRetrievalEventHandler;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a leader retrieval service which uses ZooKeeper to retrieve the leader information.
+	 *
+	 * @param client Client which constitutes the connection to the ZooKeeper quorum
+	 * @param retrievalPath Path of the ZooKeeper node which contains the leader information
+	 * @param leaderRetrievalEventHandler handler to notify the leader changes.
+	 */
+	public ZooKeeperLeaderRetrievalDriver(
+			CuratorFramework client,
+			String retrievalPath,
+			LeaderRetrievalEventHandler leaderRetrievalEventHandler) throws Exception {
+		this.client = checkNotNull(client, "CuratorFramework client");
+		this.cache = new NodeCache(client, retrievalPath);
+		this.retrievalPath = checkNotNull(retrievalPath);
+
+		this.leaderRetrievalEventHandler = checkNotNull(leaderRetrievalEventHandler);
+
+		client.getUnhandledErrorListenable().addListener(this);
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(connectionStateListener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception {
+		if (!running) {
+			return;
+		}
+
+		running = false;

Review comment:
       At the very beginning, I think the `volatile` is enough. But after a careful consideration, you are right. We need a lock to guard access of `running`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507382145



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -196,6 +224,16 @@ public static String getCommonStartCommand(
 		).collect(Collectors.toList());
 	}
 
+	public static Predicate<KubernetesConfigMap> getLeaderChecker() {
+		return configMap -> {
+			if (configMap.getAnnotations() != null) {
+				final String leader = configMap.getAnnotations().get(LEADER_ANNOTATION_KEY);
+				return leader != null && leader.contains(LOCK_IDENTITY);
+			}
+			return false;
+		};
+	}

Review comment:
       I will move this change to the leader elector related commits.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-714303198


   Concerning trying to cover the case where the configMaps are accidentally deleted or modified by the user, I think that in the general case our approach won't work. In the current implementation only the leader is allowed to re-create the configMap. This won't work if the configMap deletion coincides with the leader death. Even if any leader contender is allowed to re-create the configMap, then we still risk to lose all persisted information because only the dead leader knew the state. Hence, I believe we should rather fail hard if we detect a modification or deletion and tell the user because we cannot guarantee that we didn't lose relevant information.
   
   In particular to test these kind of scenarios (how does the system behave in corner cases), I believe that an integration test with K8s is actually required.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516787530



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ * run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ *
+ * <p>{@link LeaderElector#run()} is responsible for creating the leader ConfigMap and continuously update the
+ * annotation. The annotation key is {@link #LEADER_ANNOTATION_KEY} and the value is in the following json format.
+ * metadata:
+ *   annotations:
+ *     control-plane.alpha.kubernetes.io/leader: '{"holderIdentity":"623e39fb-70c3-44f1-811f-561ec4a28d75","leaseDuration":15.000000000,"acquireTime":"2020-10-20T04:06:31.431000Z","renewTime":"2020-10-22T08:51:36.843000Z","leaderTransitions":37981}'
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	@VisibleForTesting
+	public static final String LEADER_ANNOTATION_KEY = "control-plane.alpha.kubernetes.io/leader";
+
+	private final ExecutorService executorService = Executors.newSingleThreadExecutor(
+		new ExecutorThreadFactory("KubernetesLeaderElector-ExecutorService"));
+
+	public KubernetesLeaderElector(
+			NamespacedKubernetesClient kubernetesClient,
+			String namespace,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderCallbackHandler leaderCallbackHandler) {
+		super(kubernetesClient, new LeaderElectionConfigBuilder()
+			.withName(leaderConfig.getConfigMapName())
+			.withLeaseDuration(leaderConfig.getLeaseDuration())
+			.withLock(new ConfigMapLock(namespace, leaderConfig.getConfigMapName(), leaderConfig.getLockIdentity()))
+			.withRenewDeadline(leaderConfig.getRenewDeadline())
+			.withRetryPeriod(leaderConfig.getRetryPeriod())
+			.withLeaderCallbacks(new LeaderCallbacks(
+				leaderCallbackHandler::isLeader,
+				leaderCallbackHandler::notLeader,
+				newLeader -> LOG.info("New leader elected {} for {}.", newLeader, leaderConfig.getConfigMapName())
+			))
+			.build());
+		LOG.info("Create KubernetesLeaderElector {} with lock identity {}.",
+			leaderConfig.getConfigMapName(), leaderConfig.getLockIdentity());
+	}
+
+	@Override
+	public void run() {
+		CompletableFuture.runAsync(super::run, executorService);

Review comment:
       sounds good.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508595107



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java
##########
@@ -80,4 +84,14 @@
 	public static final String RESTART_POLICY_OF_NEVER = "Never";
 
 	public static final String NATIVE_KUBERNETES_COMMAND = "native-k8s";
+
+	// Constants for Kubernetes high availability
+	public static final String LEADER_ADDRESS_KEY = "address";
+	public static final String LEADER_SESSION_ID_KEY = "sessionId";
+	public static final String CHECKPOINT_COUNTER_KEY = "counter";
+	public static final String RUNNING_JOBS_REGISTRY_KEY_PREFIX = "runningJobsRegistry";
+	public static final String JOB_GRAPH_STORE_KEY_PREFIX = "jobGraph";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();

Review comment:
       I will make `LOCK_IDENTITY` non static and could be specified via `KubernetesLeaderElectionConfiguration`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507558728



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");

Review comment:
       Hmm. I get the insight from Zookeeper implementation. Actually, we have no requirement to update the suffix. I will remove the config options now.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509221909



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */

Review comment:
       I will use the composition instead.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r511276860



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionServiceTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.apache.flink.runtime.leaderelection.TestingContender;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_KEY;
+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 KubernetesLeaderElectionService}.
+ */
+public class KubernetesLeaderElectionServiceTest extends KubernetesHighAvailabilityTestBase {
+
+	@Test
+	public void testKubernetesLeaderElection() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					assertThat(leaderElectionService.getLeaderSessionID(), is(contender.getLeaderSessionID()));
+					assertThat(configMapStore.size(), is(1));
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+
+					// Revoke leader
+					leaderController.set(false);
+					contender.waitForRevokeLeader(TIMEOUT);
+					assertThat(leaderElectionService.getLeaderSessionID(), nullValue());
+					assertThat(getLeaderConfigMap().getData().size(), is(0));
+				});
+		}};
+	}
+
+	@Test
+	public void testLeaderConfigMapDeletedExternally() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					final FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> callbackHandler =
+						configMapsAndDoCallbackFuture.get(TIMEOUT, TimeUnit.MILLISECONDS);
+					assertThat(callbackHandler, is(notNullValue()));
+
+					callbackHandler.onDeleted(Collections.singletonList(configMapStore.remove(LEADER_CONFIGMAP_NAME)));
+					// The ConfigMap should be created again.
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+				});
+		}};
+	}
+
+	@Test
+	public void testLeaderConfigMapUpdatedExternally() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					final FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> callbackHandler =
+						configMapsAndDoCallbackFuture.get(TIMEOUT, TimeUnit.MILLISECONDS);
+					assertThat(callbackHandler, is(notNullValue()));
+					// Update ConfigMap with wrong data
+					getLeaderConfigMap().getData().put(Constants.LEADER_ADDRESS_KEY, "wrong data");
+					callbackHandler.onModified(Collections.singletonList(configMapStore.get(LEADER_CONFIGMAP_NAME)));
+					// The ConfigMap should be corrected
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+				});
+		}};
+	}
+
+	/**
+	 * Start multiple leaders, if current leader dies, a new one could take over and update the leader
+	 * information successfully.
+	 */
+	@Test
+	public void testMultipleLeaders() throws Exception {
+		final int leaderNum = 3;
+		final AtomicBoolean[] leaderController = new AtomicBoolean[leaderNum];
+		final TestingContender[] contenders = new TestingContender[leaderNum];
+		final KubernetesLeaderElectionService[] leaderElectionServices = new KubernetesLeaderElectionService[leaderNum];
+		for (int i = 0; i < leaderNum; i++) {
+			leaderController[i] = new AtomicBoolean(false);
+			leaderElectionServices[i] = new Context().createLeaderElectionService(leaderController[i]);
+			contenders[i] = new TestingContender(getLeaderUrl(i), leaderElectionServices[i]);
+			leaderElectionServices[i].start(contenders[i]);
+		}
+		leaderController[0].set(true);
+		contenders[0].waitForLeader(TIMEOUT);
+		assertThat(
+			configMapStore.get(LEADER_CONFIGMAP_NAME).getData().get(Constants.LEADER_ADDRESS_KEY),
+			is(getLeaderUrl(0)));
+		// Leader 0 died
+		leaderController[0].set(false);
+		contenders[0].waitForRevokeLeader(TIMEOUT);
+		// Leader 2 try to acquire
+		leaderController[2].set(true);
+		contenders[2].waitForLeader(TIMEOUT);
+		assertThat(
+			configMapStore.get(LEADER_CONFIGMAP_NAME).getData().get(Constants.LEADER_ADDRESS_KEY),
+			is(getLeaderUrl(2)));
+		for (int i = 0; i < leaderNum; i++) {
+			leaderElectionServices[i].stop();
+		}
+	}

Review comment:
       I have added a ITCase for the concurrent leader election.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516785922



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionDriver.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.leaderelection;
+
+/**
+ * A {@link LeaderElectionDriver} is responsible for performing the leader election and storing the leader information.
+ * All the leader internal state is guarded by lock in {@link LeaderElectionService}. Different driver
+ * implementations do not need to care about the lock. And it should use {@link LeaderElectionEventHandler}
+ * if it want to respond to the leader change events.
+ *
+ * <p><strong>Important</strong>: The {@link LeaderElectionDriver} could not guarantee that there is no
+ * {@link LeaderElectionEventHandler} callbacks happen after {@link #close()}.
+ */
+public interface LeaderElectionDriver {
+
+	/**
+	 * Write the current leader information to external persistent storage(e.g. Zookeeper, Kubernetes ConfigMap). This
+	 * is a blocking IO operation.
+	 *
+	 * @param leaderInformation current leader information. It could be {@link LeaderInformation#empty()}, which means
+	 * the caller want to clear the leader information on external storage.
+	 */
+	void writeLeaderInformation(LeaderInformation leaderInformation);

Review comment:
       Sounds good.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514305023



##########
File path: flink-end-to-end-tests/test-scripts/test_kubernetes_itcases.sh
##########
@@ -0,0 +1,30 @@
+#!/usr/bin/env bash
+################################################################################
+# 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.
+################################################################################
+
+source "$(dirname "$0")"/common_kubernetes.sh
+
+start_kubernetes
+
+# Set the ITCASE_KUBECONFIG environment since it is required to run the ITCases
+export ITCASE_KUBECONFIG=~/.kube/config
+
+cd $END_TO_END_DIR/../flink-kubernetes

Review comment:
       I meant to start minikube in the `MinikubeResource` or something similar. But this would have the disadvantage that we could not reuse a minikube cluster/instance for different IT cases.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509857812



##########
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:
       For example, we do the serialize/deserialize/discard-state for the job graph store or completed checkpoint. We could encounter the `Exception`. I think we could retry in such situation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508360227



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -103,6 +121,52 @@ public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, WatchC
 		return watchPodsAndDoCallbackFunction.apply(labels, podCallbackHandler);
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		configMapStore.putIfAbsent(configMap.getName(), configMap);
+		return CompletableFuture.completedFuture(null);
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final KubernetesConfigMap configMap = configMapStore.get(name);
+		if (configMap == null) {
+			return Optional.empty();
+		}
+		return Optional.of(new MockKubernetesConfigMap(configMap.getName(), new HashMap<>(configMap.getData())));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			Predicate<KubernetesConfigMap> checker,
+			FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function) {
+		return getConfigMap(configMapName).map(FunctionUtils.uncheckedFunction(
+			configMap -> {
+				final boolean shouldUpdate = checker.test(configMap);
+				if (shouldUpdate) {
+					configMapStore.put(configMap.getName(), function.apply(configMap));
+				}
+				return CompletableFuture.completedFuture(shouldUpdate);
+			}))
+			.orElseThrow(() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists."));
+	}

Review comment:
       Either `configMapStore#remove(name)` or passing in `configMapStore` requires the map to be accessed from both internal and external of the testing client. Then we would also need to deal with the concurrent access issue.
   
   I would vote for the `Function` approach, to avoid adding maintaining overhead for future efforts.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 7f34f6f31596832223b1cf2010bed1c47f7502ea Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511) 
   * 6a2602f39684ca0005350787b1c8051815acecdc UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509059229



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						}
+						return updated;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),

Review comment:
       I think this method is also lacking test coverage for exactly a case where `getConfigMap` returns `Optional.empty` or where
   
   ```
   this.internalClient.configMaps()
   									.inNamespace(namespace)
   									.createOrReplace(updatedConfigMap.getInternalResource());
   ```
   
   fails.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r511272629



##########
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:
       After a careful consideration, I think you are right. We do not need the `FunctionWithException` here. Only the update operation succeeded, we need to do the following external storage cleanup. For example, we could use `kubeClient.checkAndUpdateConfigMap(configMapName, updateFunction).whenCompleteAsync()` for this operation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508494420



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

Review comment:
       Make sense. I will introduce a `KubernetesException`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r511803649



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
##########
@@ -412,6 +415,26 @@ public RetryException(Throwable cause) {
 		}
 	}
 
+	/**
+	 * The {@link #retry(Supplier, int, Executor)} attempts will stop at this exception.
+	 */
+	public static class NotRetryException extends Exception {

Review comment:
       Just like `FutureUtils#retryWithDelay`, I have added a new parameter `Predicate<Throwable> retryPredicate` for `FutureUtils#retry`. Then it is flexible enough to control when to stop the retry.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 991bdfa5427479f845abaf91caba0069011ee9e3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874) 
   * f6be1c1045c89a0c2a2d8f1d317d41a656cae538 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509231284



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).

Review comment:
       I will add more description here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507537261



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	protected volatile LeaderContender leaderContender;
+
+	private volatile UUID issuedLeaderSessionID;
+
+	protected volatile UUID confirmedLeaderSessionID;
+
+	protected volatile String confirmedLeaderAddress;
+
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+		}
+
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		internalStop();
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (logger.isDebugEnabled()) {
+			logger.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		if (checkLeaderLatch()) {

Review comment:
       I am afraid we could not completely get rid of `checkLeaderLatch` even we adjust the `if`s. `checkLeaderLatch` is used for checking whether the real leader elector(e.g. `LeaderLatch` for ZK and `KubernetesLeaderElector` for K8s) still have the leadership. Maybe we could rename the interface to `hasLeadership` without arguments. Also we will mark `hasLeadership(UUID leaderSessionId)` as `final` so that it could not be overridden wrongly.
   
   Then the specific `LeaderElectorService` just need to implement the `hasLeadership()` via checking the real elector status.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514061417



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);

Review comment:
       Yes, `leaderElector::run` is a blocking call. It will block the thread until lost the leadership. So I will make the `ioExecutor` should be configured big. Based on this, I will introduce a dedicated single thread pool for `KubernetesLeaderElector`. And then we could support the `stop` method to shutdown the thread pool.
   
   Also the `leaderElectorRunFuture` will be unnecessary.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e33c52394024e8f1312c096a4b612d53d8848b97 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508985458



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();

Review comment:
       I will try to use the composition for the abstraction. Just like you said, it could help us to better control the `lock` and make the contract more clear.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509278960



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

Review comment:
       Actually, the `JobManager`s here mean the whole process. I will refine the description after the composition.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r517218385



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory);
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		synchronized (lock) {
+			leaderContender = contender;
+			leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver(
+				this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription());
+			LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping DefaultLeaderElectionService.");
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.close();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+							"LeaderElectionService has already been stopped.", leaderSessionID);
+					}
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+							"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+					}
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			if (running) {
+				return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("hasLeadership is called after the service is stopped, returning false.");
+				}
+				return false;
+			}
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+		leaderElectionDriver.writeLeaderInformation(
+			LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress));
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the grant leadership notification since the {} has " +
+						"already been closed.", leaderElectionDriver);
+				}
+				return;
+			}
+		}
+		// The contender callback should be executed out of lock to avoid potential deadlock.
+		leaderContender.grantLeadership(issuedLeaderSessionID);
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onRevokeLeadership() {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+				}
+
+				issuedLeaderSessionID = null;
+				clearConfirmedLeaderInformation();
+
+				// Clear the old leader information on the external storage
+				leaderElectionDriver.writeLeaderInformation(LeaderInformation.empty());
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the revoke leadership notification since the {} " +
+						"has already been closed.", leaderElectionDriver);
+				}
+				return;
+			}
+		}
+		// The contender callback should be executed out of lock to avoid potential deadlock.
+		leaderContender.revokeLeadership();

Review comment:
       I am wondering whether it is potentially a problem that we only let the `leaderContender` know about the lost leadership after performing a potentially blocking operation in `leaderElectionDriver.writeLeaderInformation`. Assume that this operation takes a bit of time, then the `leaderContender` can continue working as the leader. Maybe it would be better to only try to clear the old leader information after having notified the `leaderContender` about it in particular since the clearing is not strictly required.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientITCase.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.kubeclient;
+
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client.
+ */
+public class Fabric8FlinkKubeClientITCase {
+
+	@ClassRule
+	public static KubernetesResource kubernetesResource = new KubernetesResource();
+
+	private static final String TEST_CONFIG_MAP_NAME = "test-config-map";
+
+	private static final long TIMEOUT = 120L * 1000L;
+
+	private static final Map<String, String> data = new HashMap<String, String>() {
+		{
+			put("key1", "0");
+			put("key2", "0");
+			put("key3", "0");
+		}
+	};
+
+	private FlinkKubeClient flinkKubeClient;
+
+	@Before
+	public void setup() throws Exception {
+		flinkKubeClient = kubernetesResource.getFlinkKubeClient();
+		flinkKubeClient.createConfigMap(new KubernetesConfigMap(
+			new ConfigMapBuilder()
+				.withNewMetadata()
+				.withName(TEST_CONFIG_MAP_NAME)
+				.endMetadata()
+				.withData(data)
+				.build())).get();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		flinkKubeClient.deleteConfigMap(TEST_CONFIG_MAP_NAME).get();
+	}
+
+	/**
+	 * {@link org.apache.flink.kubernetes.kubeclient.FlinkKubeClient#checkAndUpdateConfigMap} is a transactional
+	 * operation, we should definitely guarantee that the concurrent modification could work.
+	 */
+	@Test
+	public void testCheckAndUpdateConfigMapConcurrently() throws Exception {
+		// Start multiple instances to update ConfigMap concurrently
+		final List<CompletableFuture<Void>> futures = new ArrayList<>();
+		final int target = 10;
+		final int updateIntervalMs = 100;
+		for (String key : data.keySet()) {
+			futures.add(FutureUtils.runAfterwardsAsync(FutureUtils.completedVoidFuture(), () -> {

Review comment:
       I guess we could still fix this here. Very minor though.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516788405



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final FlinkKubeClient kubeClient;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		checkNotNull(leaderConfig, "Leader election configuration");
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElector.run();
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}.", this);
+		leaderElector.stop();
+		kubernetesWatch.close();
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		assert(running);
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// Get the updated ConfigMap with new leader information
+						if (confirmedLeaderAddress == null) {
+							configMap.getData().remove(LEADER_ADDRESS_KEY);
+						} else {
+							configMap.getData().put(LEADER_ADDRESS_KEY, confirmedLeaderAddress);
+						}
+						if (confirmedLeaderSessionID == null) {
+							configMap.getData().remove(LEADER_SESSION_ID_KEY);
+						} else {
+							configMap.getData().put(LEADER_SESSION_ID_KEY, confirmedLeaderSessionID.toString());
+						}
+						configMap.getLabels().putAll(configMapLabels);
+						return Optional.of(configMap);
+					}
+					return Optional.empty();
+				}).get();
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(
+					"Successfully wrote leader information: Leader={}, session ID={}.",
+					confirmedLeaderAddress,
+					confirmedLeaderSessionID);
+			}
+		} catch (Exception e) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not write leader information since ConfigMap " + configMapName
+					+ " does not exist.", e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		assert(running);
+		final Optional<KubernetesConfigMap> configMapOpt = kubeClient.getConfigMap(configMapName);
+		if (configMapOpt.isPresent()) {
+			return KubernetesLeaderElector.hasLeadership(configMapOpt.get(), lockIdentity);
+		} else {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("ConfigMap " + configMapName + " does not exist.", null));
+			return false;
+		}
+	}
+
+	private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+		@Override
+		public void isLeader() {
+			leaderElectionEventHandler.onGrantLeadership();
+		}
+
+		@Override
+		public void notLeader() {
+			leaderElectionEventHandler.onRevokeLeadership();
+			// Continue to contend the leader
+			leaderElector.run();
+		}
+	}
+
+	private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> {
+		@Override
+		public void onAdded(List<KubernetesConfigMap> configMaps) {
+			// noop
+		}
+
+		@Override
+		public void onModified(List<KubernetesConfigMap> configMaps) {
+			// We should only receive events for the watched ConfigMap
+			final KubernetesConfigMap configMap = checkConfigMaps(configMaps, configMapName);
+
+			if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+				leaderElectionEventHandler.onLeaderInformationChange(getLeaderInformationFromConfigMap(configMap));

Review comment:
       This contracts makes sense. Sorry I overlooked this piece of information.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508502536



##########
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:
       Yes. Same as above.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507615525



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java
##########
@@ -115,11 +115,7 @@ public String getImage() {
 
 	@Override
 	public Map<String, String> getCommonLabels() {
-		Map<String, String> commonLabels = new HashMap<>();
-		commonLabels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE);
-		commonLabels.put(Constants.LABEL_APP_KEY, getClusterId());
-
-		return Collections.unmodifiableMap(commonLabels);
+		return KubernetesUtils.getCommonLabels(getClusterId());

Review comment:
       Will move the abstract operation to a separate commit.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513359598



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -217,6 +230,79 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.exceptionally(
+				throwable -> {
+					throw new CompletionException(
+						new KubernetesException("Failed to create ConfigMap " + configMapName, throwable));
+				});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			Function<KubernetesConfigMap, Optional<KubernetesConfigMap>> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(
+						configMap -> function.apply(configMap).map(
+							updatedConfigMap -> {
+								try {
+									this.internalClient.configMaps()
+										.inNamespace(namespace)
+										.withName(configMapName)
+										.lockResourceVersion(updatedConfigMap.getResourceVersion())
+										.replace(updatedConfigMap.getInternalResource());
+								} catch (Throwable throwable) {
+									LOG.debug("Failed to update ConfigMap {} with data {} because of concurrent " +
+										"modifications. Trying again.", configMap.getName(), configMap.getData());
+									throw throwable;
+								}
+								return true;
+							}).orElse(false))
+					.orElseThrow(() -> new CompletionException(
+						new KubernetesException("Not retry since " + configMapName + " does not exist."))),
+				kubeClientExecutorService),
+			maxRetryAttempts,
+			// Only KubernetesClientException is retryable
+			throwable -> ExceptionUtils.findThrowable(throwable, KubernetesClientException.class).isPresent(),

Review comment:
       Nice :-)

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/itcases/Fabric8FlinkKubeClientITCase.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.itcases;
+
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client.
+ */
+public class Fabric8FlinkKubeClientITCase extends KubernetesITTestBase {
+
+	private static final String TEST_CONFIG_MAP_NAME = "test-config-map";
+
+	private static final Map<String, String> data = new HashMap<String, String>() {
+		{
+			put("key1", "0");
+			put("key2", "0");
+			put("key3", "0");
+		}
+	};
+
+	@Before
+	public void setup() throws Exception {
+		super.setup();
+		flinkKubeClient.createConfigMap(new KubernetesConfigMap(
+			new ConfigMapBuilder()
+				.withNewMetadata()
+				.withName(TEST_CONFIG_MAP_NAME)
+				.endMetadata()
+				.withData(data)
+				.build())).get();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		flinkKubeClient.deleteConfigMap(TEST_CONFIG_MAP_NAME).get();
+		super.teardown();
+	}
+
+	/**
+	 * {@link org.apache.flink.kubernetes.kubeclient.FlinkKubeClient#checkAndUpdateConfigMap} is a transactional
+	 * operation, we should definitely guarantee that the concurrent modification could work.
+	 */
+	@Test
+	public void testCheckAndUpdateConfigMapConcurrently() throws Exception {
+		// Start multiple instances to update ConfigMap concurrently
+		final List<CompletableFuture<Void>> futures = new ArrayList<>();
+		final int target = 10;
+		final int updateIntervalMs = 500;
+		data.keySet().forEach(

Review comment:
       Let's simply use a for-each loop here. KISS: Keep it simply and stupid.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/itcases/Fabric8FlinkKubeClientITCase.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.itcases;
+
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client.
+ */
+public class Fabric8FlinkKubeClientITCase extends KubernetesITTestBase {
+
+	private static final String TEST_CONFIG_MAP_NAME = "test-config-map";
+
+	private static final Map<String, String> data = new HashMap<String, String>() {
+		{
+			put("key1", "0");
+			put("key2", "0");
+			put("key3", "0");
+		}
+	};
+
+	@Before
+	public void setup() throws Exception {
+		super.setup();
+		flinkKubeClient.createConfigMap(new KubernetesConfigMap(
+			new ConfigMapBuilder()
+				.withNewMetadata()
+				.withName(TEST_CONFIG_MAP_NAME)
+				.endMetadata()
+				.withData(data)
+				.build())).get();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		flinkKubeClient.deleteConfigMap(TEST_CONFIG_MAP_NAME).get();
+		super.teardown();
+	}
+
+	/**
+	 * {@link org.apache.flink.kubernetes.kubeclient.FlinkKubeClient#checkAndUpdateConfigMap} is a transactional
+	 * operation, we should definitely guarantee that the concurrent modification could work.
+	 */
+	@Test
+	public void testCheckAndUpdateConfigMapConcurrently() throws Exception {
+		// Start multiple instances to update ConfigMap concurrently
+		final List<CompletableFuture<Void>> futures = new ArrayList<>();
+		final int target = 10;
+		final int updateIntervalMs = 500;
+		data.keySet().forEach(
+			key -> futures.add(CompletableFuture.runAsync(() -> {
+				for (int index = 0; index < target; index++) {
+					try {
+						final boolean updated = flinkKubeClient.checkAndUpdateConfigMap(
+							TEST_CONFIG_MAP_NAME,
+							configMap -> {
+								final int newValue = Integer.valueOf(configMap.getData().get(key)) + 1;
+								configMap.getData().put(key, String.valueOf(newValue));
+								return Optional.of(configMap);
+							}).get();
+						assertThat(updated, is(true));
+					} catch (Exception e) {
+						Assert.fail("Should not throw exception." + e.getMessage());

Review comment:
       I think it would be enough to simply let the exception bubble up here.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/itcases/Fabric8FlinkKubeClientITCase.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.itcases;
+
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client.
+ */
+public class Fabric8FlinkKubeClientITCase extends KubernetesITTestBase {
+
+	private static final String TEST_CONFIG_MAP_NAME = "test-config-map";
+
+	private static final Map<String, String> data = new HashMap<String, String>() {
+		{
+			put("key1", "0");
+			put("key2", "0");
+			put("key3", "0");
+		}
+	};
+
+	@Before
+	public void setup() throws Exception {
+		super.setup();
+		flinkKubeClient.createConfigMap(new KubernetesConfigMap(
+			new ConfigMapBuilder()
+				.withNewMetadata()
+				.withName(TEST_CONFIG_MAP_NAME)
+				.endMetadata()
+				.withData(data)
+				.build())).get();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		flinkKubeClient.deleteConfigMap(TEST_CONFIG_MAP_NAME).get();
+		super.teardown();
+	}
+
+	/**
+	 * {@link org.apache.flink.kubernetes.kubeclient.FlinkKubeClient#checkAndUpdateConfigMap} is a transactional
+	 * operation, we should definitely guarantee that the concurrent modification could work.
+	 */
+	@Test
+	public void testCheckAndUpdateConfigMapConcurrently() throws Exception {
+		// Start multiple instances to update ConfigMap concurrently
+		final List<CompletableFuture<Void>> futures = new ArrayList<>();
+		final int target = 10;
+		final int updateIntervalMs = 500;
+		data.keySet().forEach(
+			key -> futures.add(CompletableFuture.runAsync(() -> {
+				for (int index = 0; index < target; index++) {
+					try {
+						final boolean updated = flinkKubeClient.checkAndUpdateConfigMap(
+							TEST_CONFIG_MAP_NAME,
+							configMap -> {
+								final int newValue = Integer.valueOf(configMap.getData().get(key)) + 1;
+								configMap.getData().put(key, String.valueOf(newValue));
+								return Optional.of(configMap);
+							}).get();
+						assertThat(updated, is(true));
+					} catch (Exception e) {
+						Assert.fail("Should not throw exception." + e.getMessage());
+					}
+					try {
+						// Simulate the update interval
+						Thread.sleep(updateIntervalMs);

Review comment:
       can we lower the sleep and maybe make it random so that we get different interleavings?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516789191



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory);
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		synchronized (lock) {
+			leaderContender = contender;
+			leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver(
+				this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription());
+			LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping DefaultLeaderElectionService.");
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.close();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+							"LeaderElectionService has already been stopped.", leaderSessionID);
+					}
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+							"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+					}
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			if (running) {
+				return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("hasLeadership is called after the service is stopped, returning false.");
+				}
+				return false;
+			}
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+		leaderElectionDriver.writeLeaderInformation(
+			LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress));
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+
+				leaderContender.grantLeadership(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the grant leadership notification since the {} has " +
+						"already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onRevokeLeadership() {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+				}
+
+				issuedLeaderSessionID = null;
+				clearConfirmedLeaderInformation();
+
+				leaderContender.revokeLeadership();
+
+				// Clear the old leader information on the external storage
+				leaderElectionDriver.writeLeaderInformation(LeaderInformation.empty());

Review comment:
       ok, this makes sense. Thanks for the explanation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509289570



##########
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 + "'}";
+	}

Review comment:
       Just like zookeeper, it is for the logging. We could get the following log.
   
   ```
   2020-10-20 04:06:30,857 INFO  org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionService [] - Starting LeaderElectionService KubernetesLeaderElectionService{configMapName='k8s-ha-app2-restserver-leader'}.
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508425784



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServices.java
##########
@@ -102,146 +95,73 @@
 	/** The ZooKeeper client to use. */
 	private final CuratorFramework client;
 
-	/** The executor to run ZooKeeper callbacks on. */
-	private final Executor executor;
-
-	/** The runtime configuration. */
-	private final Configuration configuration;
-
-	/** The zookeeper based running jobs registry. */
-	private final RunningJobsRegistry runningJobsRegistry;
-
-	/** Store for arbitrary blobs. */
-	private final BlobStoreService blobStoreService;
-
 	public ZooKeeperHaServices(
 			CuratorFramework client,
 			Executor executor,
 			Configuration configuration,
 			BlobStoreService blobStoreService) {
+		super(executor, configuration, blobStoreService);
 		this.client = checkNotNull(client);
-		this.executor = checkNotNull(executor);
-		this.configuration = checkNotNull(configuration);
-		this.runningJobsRegistry = new ZooKeeperRunningJobsRegistry(client, configuration);
-
-		this.blobStoreService = checkNotNull(blobStoreService);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Services
-	// ------------------------------------------------------------------------
-
-	@Override
-	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
-	}
-
-	@Override
-	public LeaderRetrievalService getDispatcherLeaderRetriever() {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, DISPATCHER_LEADER_PATH);
 	}
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathForJobManager(jobID));
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
+		return new ZooKeeperCheckpointRecoveryFactory(client, configuration, executor);
 	}
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress) {
-		return getJobManagerLeaderRetriever(jobID);
+	public JobGraphStore getJobGraphStore() throws Exception {
+		return ZooKeeperUtils.createJobGraphs(client, configuration);
 	}
 
 	@Override
-	public LeaderRetrievalService getClusterRestEndpointLeaderRetriever() {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, REST_SERVER_LEADER_PATH);
+	public RunningJobsRegistry getRunningJobsRegistry() {
+		return new ZooKeeperRunningJobsRegistry(client, configuration);

Review comment:
       Not sure about this change.
   - This changes the behavior to create a new `ZooKeeperRunningJobsRegistry` every time.
   - This change does not belong to this commit, which is described "abstract common fields and methods"

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubeClientFactory.java
##########
@@ -71,7 +73,12 @@ public static FlinkKubeClient fromConfiguration(Configuration flinkConfig) {
 
 		final KubernetesClient client = new DefaultKubernetesClient(config);
 
-		return new Fabric8FlinkKubeClient(flinkConfig, client, KubeClientFactory::createThreadPoolForAsyncIO);
+		if (flinkKubeClient == null) {
+			flinkKubeClient = new Fabric8FlinkKubeClient(
+				flinkConfig, client, KubeClientFactory::createThreadPoolForAsyncIO);
+		}

Review comment:
       I'm not sure about this change.
   - The argument `flinkConfig` is ignored when `flinkKubeClient` is not `null`. The factory should not assume the configuration is always same when calling this method. 
   - The codes before this `if` becomes useless when `flinkKubeClient` is not `null`.
   
   I would consider reusing of the `flinkKubeClient` an optimization. If it cannot be easily achieved, I'm also ok with using separate kube clients for different component, and make this optimization a follow-up.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the pattern "{clusterId}-{componentName}-leader". Given that the cluster
+ * id is configured to "k8s-ha-app1", then we could get the following ConfigMap names.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.leader-election.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +
+				"continuously renew its lease time to indicate its existence. And the followers will do a lease " +
+				"checking against the current time. \"renewTime + leaseDuration > now\" means the leader is alive.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RENEW_DEADLINE =
+			key("high-availability.kubernetes.leader-election.renew-deadline")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Defines the deadline when the leader tries to renew the lease in ms. The leader will " +

Review comment:
       "in ms" should be removed

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

Review comment:
       Might make sense to introduce some kind of `KubernetesException`, rather than throwing `FlinkRuntimeException`.

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

Review comment:
       Could be `final`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the pattern "{clusterId}-{componentName}-leader". Given that the cluster
+ * id is configured to "k8s-ha-app1", then we could get the following ConfigMap names.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.leader-election.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +
+				"continuously renew its lease time to indicate its existence. And the followers will do a lease " +
+				"checking against the current time. \"renewTime + leaseDuration > now\" means the leader is alive.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RENEW_DEADLINE =
+			key("high-availability.kubernetes.leader-election.renew-deadline")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Defines the deadline when the leader tries to renew the lease in ms. The leader will " +
+				"give up its leadership if it cannot successfully renew the lease in the given time.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RETRY_PERIOD =
+			key("high-availability.kubernetes.leader-election.retry-period")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(5))
+			.withDescription("Defines the pause between consecutive retries in ms. All the contenders, including the " +

Review comment:
       "in ms" should be removed

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalService.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+
+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;
+
+/**
+ * The counterpart to the {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionService}.
+ * This implementation of the {@link LeaderRetrievalService} retrieves the current leader which has
+ * been elected by the {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionService}.
+ * The leader address as well as the current leader session ID is retrieved from Kubernetes ConfigMap.
+ */
+class KubernetesLeaderRetrievalService implements LeaderRetrievalService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final String configMapName;
+
+	@GuardedBy("lock")
+	private volatile String lastLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile UUID lastLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile LeaderRetrievalListener leaderListener;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private KubernetesWatch kubernetesWatch;
+
+	KubernetesLeaderRetrievalService(FlinkKubeClient kubeClient, String configMapName) {
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client should not be null.");
+		this.configMapName = checkNotNull(configMapName, "ConfigMap name should not be null.");
+
+		this.leaderListener = null;
+		this.lastLeaderAddress = null;
+		this.lastLeaderSessionID = null;
+
+		running = false;
+	}
+
+	@Override
+	public void start(LeaderRetrievalListener listener) {
+		checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "KubernetesLeaderRetrievalService can " +
+			"only be started once.");
+
+		LOG.info("Starting {}.", this);
+
+		synchronized (lock) {
+			running = true;
+			leaderListener = listener;
+			kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		}
+	}
+
+	@Override
+	public void stop() {
+		LOG.info("Stopping {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			if (kubernetesWatch != null) {
+				kubernetesWatch.close();
+			}
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "KubernetesLeaderRetrievalService{configMapName='" + configMapName + "'}";
+	}
+
+	private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> {
+
+		@Override
+		public void onAdded(List<KubernetesConfigMap> configMaps) {
+			handleEvent(configMaps);
+		}
+
+		@Override
+		public void onModified(List<KubernetesConfigMap> configMaps) {
+			handleEvent(configMaps);
+		}
+
+		@Override
+		public void onDeleted(List<KubernetesConfigMap> configMaps) {
+			// Nothing to do since a new ConfigMap will be created if it is deleted externally.
+		}
+
+		@Override
+		public void onError(List<KubernetesConfigMap> configMaps) {
+			leaderListener.handleError(new Exception("Error while watching the ConfigMap " + configMapName));
+		}
+
+		@Override
+		public void handleFatalError(Throwable throwable) {
+			leaderListener.handleError(
+				new Exception("Fatal error while watching the ConfigMap " + configMapName, throwable));
+		}
+
+		private void handleEvent(List<KubernetesConfigMap> configMaps) {
+			synchronized (lock) {
+				if (running) {
+					configMaps.forEach(e -> {
+						if (e.getName().equals(configMapName)) {

Review comment:
       Should we add an assertion on the config map name?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the pattern "{clusterId}-{componentName}-leader". Given that the cluster
+ * id is configured to "k8s-ha-app1", then we could get the following ConfigMap names.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.leader-election.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +

Review comment:
       "in ms" should be removed

##########
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:
       Should we also add an assertion on the confip map name for the delete events?

##########
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)) {

Review comment:
       Shouldn't we check the config map name before checking whether it is updated externally?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	protected static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	protected static final String LEADER_ANNOTATION_KEY = "control-plane.alpha.kubernetes.io/leader";

Review comment:
       Should annotate with `@VisibleForTesting`.

##########
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)) {

Review comment:
       Why would we receive events for config map with another name? Shouldn't we only received events for the config map that we watch?
   Maybe we should add an assertion for the config map name here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r511878028



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +231,73 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.exceptionally(throwable -> {
+				if (throwable != null) {
+					throw new CompletionException(
+						new KubernetesException("Failed to create ConfigMap " + configMapName, throwable));
+				}
+				return null;
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {

Review comment:
       Copy the comment above.
   
   We do not need the FunctionWithException here. Only the update operation succeeded, we need to do the following external storage cleanup. For example, we could use kubeClient.checkAndUpdateConfigMap(configMapName, updateFunction).whenCompleteAsync() for this operation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 5e2cfa21731b9fb5dd976cce821bfda78daa91b7 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 37efc6f14c55e2bd261b70924c9525c6539447f8 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420) 
   * 7f34f6f31596832223b1cf2010bed1c47f7502ea Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509938404



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						}
+						return updated;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),

Review comment:
       Ah ok, now I understand the problem. Thanks for the clarification.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 removed a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 removed a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-713591200


   @tillrohrmann I have gone though all your comments and will update the PR soon. Just two quick points need to confirm with you.
   
   * Do you think we should not handle the externally deletion/update? For example, a Flink cluster with HA configured is running, some user delete/update the ConfigMap via `kubectl`. If it is yes, I will remove the operations in `KubernetesLeaderElectionService#Watcher`. And change some "ConfigMap not exists" behavior.
   * I am afraid it is hard to use a real K8s server in the UT because it is not very easy to start a `minikube`. I will try to add the unit tests for the contract testing now and leave the real cluster test in E2E test implementation. Does it make sense?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 37efc6f14c55e2bd261b70924c9525c6539447f8 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420) 
   * 7f34f6f31596832223b1cf2010bed1c47f7502ea UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508997664



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	protected static final String LOCK_IDENTITY = UUID.randomUUID().toString();

Review comment:
       IIUC, each Flink cluster will have separate cluster-id. So they will have different ConfigMap name just like following.
   * k8s-ha-app1-restserver-leader
   * k8s-ha-app2-restserver-leader
   
   For multiple JobManagers in a Flink cluster(session mode), each JobManager will have a separate ConfigMap binding to the job id.
   * k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
   * k8s-ha-app1-00000000000000000000000000000002-jobmanager-leader
   
   Both the situation, I think one `LOCK_IDENTITY` could work. But follow xintong's suggestion, I will make it could be specified via `KubernetesLeaderElectionConfiguration`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-719267643


   @tillrohrmann I really appreciate for your round after round reviews. Unfortunately, we introduce a big refactor and the PR grows bigger and bigger. For the easier review, I have reorg the commits.
   * The ZooKeeper leader election/retrieval service refactor has been split into three commits.
       - [hotfix][test] Refactor TestingContender and TestingListener to abstract some common methods
       - [hotfix][coordination] Introduce new interfaces and data structure for LeaderElection and LeaderRetrieval composition mechanism
       - [hotfix][coordination] Wire ZooKeeper leader election/retrieval services to new interface
   * Kubernetes related changes has been split into three commits
       - [FLINK-19542][k8s] Introduce data structures and common operations for KubernetesConfigMap
       - [FLINK-19542][k8s] Introduce data structures and interface for KubernetesLeaderElector
       - [FLINK-19542][k8s] Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API
   
   For the tests, I really like the idea you have suggested that test components in layers. I have tried to do this as guideline.
   * Test `DefaultLeaderElectionService` and `DefaultLeaderRetrievalService` with testing driver and testing event handler
   * ZooKeeper
       * Migrate the current tests to directly use driver
   * Kubernetes
       * Test ConfigMap related interfaces, especially for our most important interface `checkAndUpdateConfigMap`. It should work concurrently.
       * Test `KubernetesLeaderElector`. It should support multiple contenders.
       * Test `KubernetesLeaderElectionDriver` and `KubernetesLeaderRetrievalDriver` directly.
       * A simple integration test for leader election and retrieval on real K8s


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509939287



##########
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:
       Yes. You are right. What I mean is that we could also add some individual tests for HA service, not just a full Flink cluster with K8s ha enabled. They will be implemented in the E2E module. I have left a comment in FLINK-19545.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8720",
       "triggerID" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753",
       "triggerID" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884",
       "triggerID" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8956",
       "triggerID" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8956) 
   * e2795d9209fa2241cf0909737df9dca17defea4e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 1e31aa2c0c7217ba82db789933651036d2fa7e2d Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385) 
   * 37efc6f14c55e2bd261b70924c9525c6539447f8 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e33c52394024e8f1312c096a4b612d53d8848b97 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653) 
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516786301



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+
+		client.getConnectionStateListenable().removeListener(listener);
+
+		Exception exception = null;
+
+		try {
+			cache.close();
+		} catch (Exception e) {
+			exception = e;
+		}
+
+		try {
+			leaderLatch.close();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		assert(running);
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		leaderElectionEventHandler.onGrantLeadership();
+	}
+
+	@Override
+	public void notLeader() {
+		leaderElectionEventHandler.onRevokeLeadership();
+	}
+
+	@Override
+	public void nodeChanged() throws Exception {
+		if (leaderLatch.hasLeadership()) {
+			ChildData childData = cache.getCurrentData();
+			if (childData != null) {
+				final byte[] data = childData.getData();
+				if (data != null && data.length > 0) {
+					final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+					final ObjectInputStream ois = new ObjectInputStream(bais);
+
+					final String leaderAddress = ois.readUTF();
+					final UUID leaderSessionID = (UUID) ois.readObject();
+
+					leaderElectionEventHandler.onLeaderInformationChange(
+						LeaderInformation.known(leaderSessionID, leaderAddress));
+					return;
+				}
+			}
+			leaderElectionEventHandler.onLeaderInformationChange(LeaderInformation.empty());
+		}
+	}
+
+	/**
+	 * Writes the current leader's address as well the given leader session ID to ZooKeeper.
+	 */
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		assert(running);
+		// this method does not have to be synchronized because the curator framework client
+		// is thread-safe. We do not write the empty data to ZooKeeper here. Because check-leadership-and-update
+		// is not a transactional operation. We may wrongly clear the data written by new leader.
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Write leader information: {}.", leaderInformation);
+		}
+		if (leaderInformation.equals(LeaderInformation.empty())) {
+			return;

Review comment:
       The described behaviour sounds good to me @wangyang0918.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * aad1ba94672d89de5a4fc814297f0c40e3552db0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293) 
   * 1e31aa2c0c7217ba82db789933651036d2fa7e2d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 1e31aa2c0c7217ba82db789933651036d2fa7e2d Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509942532



##########
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:
       Wouldn't this be covered by the `onModified` callback which the leader will receive? The leader should then try to re-create the lost information.
   
   This actually brings me to another question: What happens if the configMap is being deleted by the user and at the same time the current leader dies for some reason? Who would then re-create the configMap? And even if we are able to recreate the configMap, how would we be able to recover the old state (e.g. all the submitted jobs)?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507358801



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -104,6 +107,67 @@ KubernetesWatch watchPodsAndDoCallback(
 		Map<String, String> labels,
 		WatchCallbackHandler<KubernetesPod> podCallbackHandler);
 
+	/**
+	 * Create the ConfigMap with specified content. If the ConfigMap already exists, nothing will happen.
+	 *
+	 * @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 empty if the ConfigMap does not exist.
+	 */
+	Optional<KubernetesConfigMap> getConfigMap(String name);
+
+	/**
+	 * Update an existing ConfigMap with the data.
+	 *
+	 * @param configMapName ConfigMap to be replaced with. 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_MAX_RETRY_ATTEMPTS}.
+	 * @param checker       Only the checker return true, the ConfigMap will be updated.
+	 * @param function      The obtained ConfigMap will be applied to this function and get a new one to replace.
+	 *
+	 * @return Return the ConfigMap update future.
+	 */
+	CompletableFuture<Boolean> checkAndUpdateConfigMap(
+		String configMapName,
+		Predicate<KubernetesConfigMap> checker,
+		FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function);

Review comment:
       Hmm. This is a useful suggestion. I agree with you that returning an `Optional<KubernetesConfigMap>` could make the contract easier to use.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513211116



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalDriver.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.leaderretrieval;
+
+/**
+ * A {@link LeaderRetrievalDriver} is responsible for retrieves the current leader which has been elected by the
+ * {@link org.apache.flink.runtime.leaderelection.LeaderElectionDriver}.
+ */
+public interface LeaderRetrievalDriver extends AutoCloseable {
+
+	/**
+	 * Close the services used for leader retrieval.
+	 */
+	void close() throws Exception;

Review comment:
       I will make `LeaderRetrievalDriver` and `LeaderElectionDriver` not extend from `AutoCloseable`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8720",
       "triggerID" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753",
       "triggerID" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884",
       "triggerID" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 2317c84938b7434c743b13a3f478d301a4c70443 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753) 
   * ed02e962a354185fb37eaedbdfee217d691bd16b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 991bdfa5427479f845abaf91caba0069011ee9e3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874) 
   * f6be1c1045c89a0c2a2d8f1d317d41a656cae538 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514148384



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Base class for high availability unit tests with a configured testing Kubernetes client.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {

Review comment:
       Make sense. It is a little difficult to understand the current `MockKubernetesLeaderElector` implementation. I will introduce a utility to capture `LeaderCallbackHandler`. Then we could call the `isLeader/notLeader` manually. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * f6be1c1045c89a0c2a2d8f1d317d41a656cae538 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507403885



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,58 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateAndDeleteConfigMap() {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(haLabels);
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+
+		final Supplier<Exception> configMapNotExistException = () -> new Exception("ConfigMap not exist");
+		FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function = c -> {
+			c.getData().put(LEADER_ADDRESS_KEY, LEADER_ADDRESS_NEW);
+			return c;
+		};
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+
+		// Checker not pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(LEADER_CONFIG_MAP_NAME, c -> false, function).get();
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(LEADER_CONFIG_MAP_NAME, c -> true, function).get();
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS_NEW));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+	}

Review comment:
       Actually, I change the implementation from `assertThat(configMapOpt.isPresent(), true);` to throw `configMapNotExistException`. I could get them back and split into two cases.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507566899



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.client.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(30))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +
+				"continuously renew its lease time to indicate its existence. And the followers will do a lease " +
+				"checking against the current time. \"renewTime + leaseDuration > now\" means the leader is alive.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RENEW_DEADLINE =
+			key("high-availability.kubernetes.client.renew-deadline")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Defines the deadline when the leader tries to renew the lease in ms. If it could not " +
+				"succeed in the given time, the renew operation will be aborted.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RETRY_PERIOD =
+			key("high-availability.kubernetes.client.retry-period")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(3))

Review comment:
       Yes, the default value of `KubernetesLeaderElector` need to be optimized. I will use the following default values.
   * lease-duration: 15s
   * renew-deadline: 15s
   * retry-period: 5s
   
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509046852



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	protected static final String LOCK_IDENTITY = UUID.randomUUID().toString();

Review comment:
       But what if we have multiple standby `JobManagers` for the same job in the `MiniCluster`? I think this can be the case when using the `TestingMiniCluster`.
   
   I guess what I wanna say is that every `KubernetesLeaderElector` should have its own identity token. I am not so sure whether this should be configurable, though. The problem I see with this option is that people will accidently pick the same token for different electors.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514297726



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Base class for high availability unit tests with a configured testing Kubernetes client.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+	private static final String CLUSTER_ID = "leader-test-cluster";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	public static final String LEADER_URL = "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher";
+	public static final String LEADER_CONFIGMAP_NAME = "leader-test-cluster";
+
+	protected static final long TIMEOUT = 30L * 1000L;
+
+	protected ExecutorService executorService;
+	protected Configuration configuration;
+
+	@Before
+	public void setup() {
+		configuration = new Configuration();
+		configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
+		executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	}
+
+	@After
+	public void teardown() throws Exception {
+		executorService.shutdownNow();
+		executorService.awaitTermination(TIMEOUT, TimeUnit.MILLISECONDS);
+	}
+
+	/**
+	 * Context to leader election and retrieval tests.
+	 */
+	protected class Context {
+		/**
+		 * The configMapStore should only be used for set the following functions. Use a ConcurrentHashMap here since it
+		 * could be updated by {@link TestingFlinkKubeClient.MockKubernetesLeaderElector} and
+		 * {@link KubernetesLeaderElectionDriver}.
+		 */
+		private final Map<String, KubernetesConfigMap> configMapStore = new ConcurrentHashMap<>();
+
+		final AtomicBoolean leaderController = new AtomicBoolean(false);
+
+		final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+			configMapCallbackFutures = new ArrayList<>();
+
+		final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture = new CompletableFuture<>();
+		final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+		final FlinkKubeClient flinkKubeClient;
+
+		final LeaderElectionDriver leaderElectionDriver;
+		final TestingLeaderElectionEventHandler electionEventHandler;
+
+		final LeaderRetrievalDriver leaderRetrievalDriver;
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler;
+
+		Context() {
+			flinkKubeClient = getFlinkKubeClient();
+
+			electionEventHandler = new TestingLeaderElectionEventHandler(LEADER_URL);
+			leaderElectionDriver = createLeaderElectionDriver();
+
+			retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+			leaderRetrievalDriver = createLeaderRetrievalDriver();
+		}
+
+		void runTestAndGrantLeadershipToContender(RunnableWithException testMethod) throws Exception {
+			// Grant leadership
+			leaderController.set(true);
+			electionEventHandler.waitForLeader(TIMEOUT);
+			assertThat(electionEventHandler.isLeader(), is(true));
+
+			testMethod.run();
+
+			leaderElectionDriver.close();
+			leaderRetrievalDriver.close();
+		}
+
+		KubernetesConfigMap getLeaderConfigMap() {
+			final Optional<KubernetesConfigMap> configMapOpt = flinkKubeClient.getConfigMap(LEADER_CONFIGMAP_NAME);
+			assertThat(configMapOpt.isPresent(), is(true));
+			return configMapOpt.get();
+		}
+
+		protected FlinkKubeClient getFlinkKubeClient() {
+			return TestingFlinkKubeClient.builder()
+				.setCreateConfigMapFunction(configMap -> {
+					configMapStore.put(configMap.getName(), configMap);
+					return CompletableFuture.completedFuture(null);
+				})
+				.setGetConfigMapFunction(configMapName -> Optional.ofNullable(configMapStore.get(configMapName)))
+				.setCheckAndUpdateConfigMapFunction((configMapName, updateFunction) -> {
+					final KubernetesConfigMap configMap = configMapStore.get(configMapName);
+					if (configMap != null) {
+						try {
+							final boolean updated = updateFunction.apply(configMap).map(
+								updateConfigMap -> {
+									configMapStore.put(configMap.getName(), updateConfigMap);
+									return true;
+								}).orElse(false);
+							return CompletableFuture.completedFuture(updated);
+						} catch (Throwable throwable) {
+							throw new CompletionException(new KubernetesException("Failed to execute update fuction"));
+						}
+					}
+					throw new CompletionException(
+						new KubernetesException("ConfigMap " + configMapName + " does not exists."));
+				})
+				.setWatchConfigMapsFunction((ignore, handler) -> {
+					final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> future =
+						new CompletableFuture<>();
+					future.complete(handler);
+					configMapCallbackFutures.add(future);
+					return new TestingFlinkKubeClient.MockKubernetesWatch();
+				})
+				.setDeleteConfigMapFunction(name -> {
+					configMapStore.remove(name);
+					return FutureUtils.completedVoidFuture();
+				})
+				.setDeleteConfigMapByLabelFunction(labels -> {
+					deleteConfigMapByLabelsFuture.complete(labels);
+					return FutureUtils.completedVoidFuture();
+				})
+				.setCloseConsumer(closeKubeClientFuture::complete)
+				.setLeaderController(leaderController)
+				.build();
+		}
+
+		private LeaderElectionDriver createLeaderElectionDriver() {
+			final KubernetesLeaderElectionConfiguration leaderConfig = new KubernetesLeaderElectionConfiguration(
+				LEADER_CONFIGMAP_NAME, LOCK_IDENTITY, configuration);
+			final KubernetesLeaderElectionDriverFactory factory = new KubernetesLeaderElectionDriverFactory(
+				flinkKubeClient, executorService, leaderConfig);
+			return factory.createLeaderElectionDriver(
+				electionEventHandler, electionEventHandler::handleError, LEADER_URL);
+		}
+
+		private LeaderRetrievalDriver createLeaderRetrievalDriver() {
+			final KubernetesLeaderRetrievalDriverFactory factory = new KubernetesLeaderRetrievalDriverFactory(
+				flinkKubeClient, LEADER_CONFIGMAP_NAME);
+			return factory.createLeaderRetrievalDriver(retrievalEventHandler, retrievalEventHandler::handleError);
+		}
+	}
+
+	public static DefaultLeaderElectionService createLeaderElectionService(
+			Configuration configuration,
+			FlinkKubeClient kubeClient,
+			String configMapName,
+			String lockIdentity,
+			ExecutorService executorService) {
+		final KubernetesLeaderElectionConfiguration leaderConfig = new KubernetesLeaderElectionConfiguration(
+			configMapName, lockIdentity, configuration);
+		return new DefaultLeaderElectionService(new KubernetesLeaderElectionDriverFactory(
+			kubeClient, executorService, leaderConfig));
+	}
+
+	public static DefaultLeaderRetrievalService createLeaderRetrievalService(
+			FlinkKubeClient flinkKubeClient, String configMapName) {
+		return new DefaultLeaderRetrievalService(
+			new KubernetesLeaderRetrievalDriverFactory(flinkKubeClient, configMapName));
+	}

Review comment:
       Will removed after refactor the 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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507376641



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,68 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		return CompletableFuture.runAsync(
+			() -> {
+				if (!getConfigMap(configMap.getName()).isPresent()) {
+					this.internalClient.configMaps().create(configMap.getInternalResource());
+				}
+			},
+			kubeClientExecutorService);
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			Predicate<KubernetesConfigMap> checker,
+			FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean shouldUpdate = checker.test(configMap);
+						if (!shouldUpdate) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						} else {
+							this.internalClient.configMaps()
+								.inNamespace(namespace)
+								.createOrReplace(function.apply(configMap).getInternalResource());
+						}
+						return shouldUpdate;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),
+				kubeClientExecutorService),
+			maxRetryAttempts,
+			kubeClientExecutorService);
+	}

Review comment:
       Yes.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516404322



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java
##########
@@ -160,6 +161,39 @@ public void testRetryCancellation() throws Exception {
 		}
 	}
 
+	/**
+	 * Test that {@link FutureUtils#retry} should stop at non-retryable exception.
+	 */
+	@Test
+	public void testStopAtNonRetryableException() {
+		final int retries = 10;
+		final int notRetry = 3;
+		final AtomicInteger atomicInteger = new AtomicInteger(0);
+		final String notRetryExceptionMsg = "Non-retryable exception";
+		CompletableFuture<Boolean> retryFuture = FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> {
+					if (atomicInteger.incrementAndGet() == notRetry) {
+						// throw non-retryable exception
+						throw new CompletionException(new FlinkRuntimeException(notRetryExceptionMsg));
+					} else {
+						throw new CompletionException(new FlinkException("Test exception"));
+					}
+				},
+				TestingUtils.defaultExecutor()),
+			retries,
+			throwable -> ExceptionUtils.findThrowable(throwable, FlinkException.class).isPresent(),
+			TestingUtils.defaultExecutor());
+
+		try {
+			retryFuture.get();
+			fail("Exception should be thrown.");
+		} catch (Exception ex) {
+			assertThat(ExceptionUtils.findThrowableWithMessage(ex, notRetryExceptionMsg).isPresent(), is(true));

Review comment:
       Nice suggestion. I will change other similar exception `assert`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory);
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		synchronized (lock) {
+			leaderContender = contender;
+			leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver(
+				this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription());
+			LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping DefaultLeaderElectionService.");
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.close();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+							"LeaderElectionService has already been stopped.", leaderSessionID);
+					}
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+							"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+					}
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			if (running) {
+				return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("hasLeadership is called after the service is stopped, returning false.");
+				}
+				return false;
+			}
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+		leaderElectionDriver.writeLeaderInformation(
+			LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress));
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+
+				leaderContender.grantLeadership(issuedLeaderSessionID);

Review comment:
       Yes. We may have a potential deadlock here. And callback of `leaderContender` should not happen under the `lock`. I will move all the callback out of `lock`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionDriver.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.leaderelection;
+
+/**
+ * A {@link LeaderElectionDriver} is responsible for performing the leader election and storing the leader information.
+ * All the leader internal state is guarded by lock in {@link LeaderElectionService}. Different driver
+ * implementations do not need to care about the lock. And it should use {@link LeaderElectionEventHandler}
+ * if it want to respond to the leader change events.
+ *
+ * <p><strong>Important</strong>: The {@link LeaderElectionDriver} could not guarantee that there is no
+ * {@link LeaderElectionEventHandler} callbacks happen after {@link #close()}.
+ */
+public interface LeaderElectionDriver {
+
+	/**
+	 * Write the current leader information to external persistent storage(e.g. Zookeeper, Kubernetes ConfigMap). This
+	 * is a blocking IO operation.
+	 *
+	 * @param leaderInformation current leader information. It could be {@link LeaderInformation#empty()}, which means
+	 * the caller want to clear the leader information on external storage.
+	 */
+	void writeLeaderInformation(LeaderInformation leaderInformation);

Review comment:
       `writeLeaderInformation` will only take effect when the current driver still have the leadership. The caller do not need to make sure of it. I will add this to the interface description.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionDriver.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.leaderelection;
+
+/**
+ * A {@link LeaderElectionDriver} is responsible for performing the leader election and storing the leader information.
+ * All the leader internal state is guarded by lock in {@link LeaderElectionService}. Different driver
+ * implementations do not need to care about the lock. And it should use {@link LeaderElectionEventHandler}
+ * if it want to respond to the leader change events.
+ *
+ * <p><strong>Important</strong>: The {@link LeaderElectionDriver} could not guarantee that there is no
+ * {@link LeaderElectionEventHandler} callbacks happen after {@link #close()}.
+ */
+public interface LeaderElectionDriver {
+
+	/**
+	 * Write the current leader information to external persistent storage(e.g. Zookeeper, Kubernetes ConfigMap). This
+	 * is a blocking IO operation.
+	 *
+	 * @param leaderInformation current leader information. It could be {@link LeaderInformation#empty()}, which means
+	 * the caller want to clear the leader information on external storage.
+	 */
+	void writeLeaderInformation(LeaderInformation leaderInformation);

Review comment:
       "The write operation takes effect only when the driver still has the leadership." The caller do not need to make sure of it. I will add this to the interface description.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+
+		client.getConnectionStateListenable().removeListener(listener);
+
+		Exception exception = null;
+
+		try {
+			cache.close();
+		} catch (Exception e) {
+			exception = e;
+		}
+
+		try {
+			leaderLatch.close();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		assert(running);
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		leaderElectionEventHandler.onGrantLeadership();
+	}
+
+	@Override
+	public void notLeader() {
+		leaderElectionEventHandler.onRevokeLeadership();
+	}
+
+	@Override
+	public void nodeChanged() throws Exception {
+		if (leaderLatch.hasLeadership()) {
+			ChildData childData = cache.getCurrentData();
+			if (childData != null) {
+				final byte[] data = childData.getData();
+				if (data != null && data.length > 0) {
+					final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+					final ObjectInputStream ois = new ObjectInputStream(bais);
+
+					final String leaderAddress = ois.readUTF();
+					final UUID leaderSessionID = (UUID) ois.readObject();
+
+					leaderElectionEventHandler.onLeaderInformationChange(
+						LeaderInformation.known(leaderSessionID, leaderAddress));
+					return;
+				}
+			}
+			leaderElectionEventHandler.onLeaderInformationChange(LeaderInformation.empty());
+		}
+	}
+
+	/**
+	 * Writes the current leader's address as well the given leader session ID to ZooKeeper.
+	 */
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		assert(running);
+		// this method does not have to be synchronized because the curator framework client
+		// is thread-safe. We do not write the empty data to ZooKeeper here. Because check-leadership-and-update
+		// is not a transactional operation. We may wrongly clear the data written by new leader.
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Write leader information: {}.", leaderInformation);
+		}
+		if (leaderInformation.equals(LeaderInformation.empty())) {
+			return;

Review comment:
       I think the normal behavior should be that once the current `LeaderElectionDriver` is not the leader, it is responsible for cleaning up the leader information on the external storage(Kubernetes, ZooKeeper). And it should only be executed when a new leader has not write his leader information.
   
   For ZooKeeper, we are using an ephemeral node for leader information and it will be cleaned up automatically when ZooKeeper client timeout. So it is reasonable to not do the clean up here.
   
   However, for Kubernetes, we should and could do this. Since check the leadership(aka check the annotation on the leader ConfigMap) and remove leader information is a transactional operation, we could guarantee that we will not overwrite a new leader information.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+
+		client.getConnectionStateListenable().removeListener(listener);
+
+		Exception exception = null;
+
+		try {
+			cache.close();
+		} catch (Exception e) {
+			exception = e;
+		}
+
+		try {
+			leaderLatch.close();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		assert(running);
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		leaderElectionEventHandler.onGrantLeadership();
+	}
+
+	@Override
+	public void notLeader() {
+		leaderElectionEventHandler.onRevokeLeadership();
+	}
+
+	@Override
+	public void nodeChanged() throws Exception {
+		if (leaderLatch.hasLeadership()) {
+			ChildData childData = cache.getCurrentData();
+			if (childData != null) {
+				final byte[] data = childData.getData();
+				if (data != null && data.length > 0) {
+					final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+					final ObjectInputStream ois = new ObjectInputStream(bais);
+
+					final String leaderAddress = ois.readUTF();
+					final UUID leaderSessionID = (UUID) ois.readObject();
+
+					leaderElectionEventHandler.onLeaderInformationChange(
+						LeaderInformation.known(leaderSessionID, leaderAddress));
+					return;
+				}
+			}
+			leaderElectionEventHandler.onLeaderInformationChange(LeaderInformation.empty());
+		}
+	}
+
+	/**
+	 * Writes the current leader's address as well the given leader session ID to ZooKeeper.
+	 */
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		assert(running);
+		// this method does not have to be synchronized because the curator framework client
+		// is thread-safe. We do not write the empty data to ZooKeeper here. Because check-leadership-and-update
+		// is not a transactional operation. We may wrongly clear the data written by new leader.
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Write leader information: {}.", leaderInformation);
+		}
+		if (leaderInformation.equals(LeaderInformation.empty())) {
+			return;

Review comment:
       I will add more description on the interface `LeaderElectionDriver#writeLeaderInformation`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
##########
@@ -520,6 +519,47 @@ public void testEphemeralZooKeeperNodes() throws Exception {
 		}
 	}
 
+	@Test
+	public void testNotLeader() throws Exception {
+
+		final TestingLeaderElectionEventHandler electionEventHandler =
+			new TestingLeaderElectionEventHandler(TEST_LEADER);
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+		LeaderElectionDriver leaderElectionDriver = null;
+		LeaderRetrievalDriver leaderRetrievalDriver = null;
+		try {
+
+			leaderElectionDriver =
+				ZooKeeperUtils.createLeaderElectionDriverFactory(client, configuration)
+					.createLeaderElectionDriver(electionEventHandler, electionEventHandler::handleError, TEST_URL);
+			electionEventHandler.init(leaderElectionDriver);
+
+			electionEventHandler.waitForLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(TEST_LEADER));
+
+			// Leader is revoked
+			((ZooKeeperLeaderElectionDriver) leaderElectionDriver).notLeader();
+			electionEventHandler.waitForRevokeLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LeaderInformation.empty()));
+			// The data on ZooKeeper it not be cleared

Review comment:
       Since in the currently ZooKeeper HA implementation, check the leadership and then update the ZooKeeper node(aka clean up the leader information here) is not a transactional operation. So we may wrongly update the ZooKeeper node with empty data. What I have done is do nothing and the ephemeral node disappear automatically.
   
   I add this test to verify this behavior and make sure when the behavior changed we could find it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
##########
@@ -520,6 +519,47 @@ public void testEphemeralZooKeeperNodes() throws Exception {
 		}
 	}
 
+	@Test
+	public void testNotLeader() throws Exception {
+
+		final TestingLeaderElectionEventHandler electionEventHandler =
+			new TestingLeaderElectionEventHandler(TEST_LEADER);
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+		LeaderElectionDriver leaderElectionDriver = null;
+		LeaderRetrievalDriver leaderRetrievalDriver = null;
+		try {
+
+			leaderElectionDriver =
+				ZooKeeperUtils.createLeaderElectionDriverFactory(client, configuration)
+					.createLeaderElectionDriver(electionEventHandler, electionEventHandler::handleError, TEST_URL);
+			electionEventHandler.init(leaderElectionDriver);
+
+			electionEventHandler.waitForLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(TEST_LEADER));
+
+			// Leader is revoked
+			((ZooKeeperLeaderElectionDriver) leaderElectionDriver).notLeader();
+			electionEventHandler.waitForRevokeLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LeaderInformation.empty()));
+			// The data on ZooKeeper it not be cleared

Review comment:
       Since in the current ZooKeeper HA implementation, check the leadership and then update the ZooKeeper node(aka clean up the leader information here) is not a transactional operation. So we may wrongly update the ZooKeeper node with empty data. What I have done is do nothing and the ephemeral node disappear automatically.
   
   I add this test to verify this behavior and make sure when the behavior changed we could find it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
##########
@@ -520,6 +519,47 @@ public void testEphemeralZooKeeperNodes() throws Exception {
 		}
 	}
 
+	@Test
+	public void testNotLeader() throws Exception {
+
+		final TestingLeaderElectionEventHandler electionEventHandler =
+			new TestingLeaderElectionEventHandler(TEST_LEADER);
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+		LeaderElectionDriver leaderElectionDriver = null;
+		LeaderRetrievalDriver leaderRetrievalDriver = null;
+		try {
+
+			leaderElectionDriver =
+				ZooKeeperUtils.createLeaderElectionDriverFactory(client, configuration)
+					.createLeaderElectionDriver(electionEventHandler, electionEventHandler::handleError, TEST_URL);
+			electionEventHandler.init(leaderElectionDriver);
+
+			electionEventHandler.waitForLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(TEST_LEADER));
+
+			// Leader is revoked
+			((ZooKeeperLeaderElectionDriver) leaderElectionDriver).notLeader();
+			electionEventHandler.waitForRevokeLeader(timeout);
+			assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LeaderInformation.empty()));
+			// The data on ZooKeeper it not be cleared

Review comment:
       Since in the current ZooKeeper HA implementation, check the leadership and then update the ZooKeeper node(aka clean up the leader information here) is not a transactional operation. So we may wrongly update the ZooKeeper node with empty data. What I have done is doing nothing and the ephemeral node disappear automatically.
   
   I add this test to verify this behavior and make sure when the behavior changed we could find it.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientITCase.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.kubeclient;
+
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client.
+ */
+public class Fabric8FlinkKubeClientITCase {
+
+	@ClassRule
+	public static KubernetesResource kubernetesResource = new KubernetesResource();
+
+	private static final String TEST_CONFIG_MAP_NAME = "test-config-map";
+
+	private static final long TIMEOUT = 120L * 1000L;
+
+	private static final Map<String, String> data = new HashMap<String, String>() {
+		{
+			put("key1", "0");
+			put("key2", "0");
+			put("key3", "0");
+		}
+	};
+
+	private FlinkKubeClient flinkKubeClient;
+
+	@Before
+	public void setup() throws Exception {
+		flinkKubeClient = kubernetesResource.getFlinkKubeClient();
+		flinkKubeClient.createConfigMap(new KubernetesConfigMap(
+			new ConfigMapBuilder()
+				.withNewMetadata()
+				.withName(TEST_CONFIG_MAP_NAME)
+				.endMetadata()
+				.withData(data)
+				.build())).get();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		flinkKubeClient.deleteConfigMap(TEST_CONFIG_MAP_NAME).get();
+	}
+
+	/**
+	 * {@link org.apache.flink.kubernetes.kubeclient.FlinkKubeClient#checkAndUpdateConfigMap} is a transactional
+	 * operation, we should definitely guarantee that the concurrent modification could work.
+	 */
+	@Test
+	public void testCheckAndUpdateConfigMapConcurrently() throws Exception {
+		// Start multiple instances to update ConfigMap concurrently
+		final List<CompletableFuture<Void>> futures = new ArrayList<>();
+		final int target = 10;
+		final int updateIntervalMs = 100;
+		for (String key : data.keySet()) {
+			futures.add(FutureUtils.runAfterwardsAsync(FutureUtils.completedVoidFuture(), () -> {

Review comment:
       Because we assume to increase the number one by one and check the target finally. Then we have to call `flinkKubeClient.checkAndUpdateConfigMap().get` and it will throw exception here. We need to execute a `RunnableWithException` not `Runnable`. Right?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +271,131 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateConfigMap() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+		final Optional<KubernetesConfigMap> currentOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(currentOpt.isPresent(), is(true));
+		assertThat(currentOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	@Test
+	public void testCreateConfigMapAlreadyExisting() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+
+		mockCreateConfigMapAlreadyExisting(configMap.getInternalResource());
+		configMap.getData().put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_NEW_VALUE);
+		try {
+			this.flinkKubeClient.createConfigMap(configMap).get();
+			fail("Exception should be thrown.");
+		} catch (Exception ex) {
+			assertThat(ex.getMessage(),
+				containsString("Failed to create ConfigMap " + TESTING_CONFIG_MAP_NAME));

Review comment:
       I will fix all the `assert` with exception in this pattern in this PR and following PRs.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ * run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ *
+ * <p>{@link LeaderElector#run()} is responsible for creating the leader ConfigMap and continuously update the
+ * annotation. The annotation key is {@link #LEADER_ANNOTATION_KEY} and the value is in the following json format.
+ * metadata:
+ *   annotations:
+ *     control-plane.alpha.kubernetes.io/leader: '{"holderIdentity":"623e39fb-70c3-44f1-811f-561ec4a28d75","leaseDuration":15.000000000,"acquireTime":"2020-10-20T04:06:31.431000Z","renewTime":"2020-10-22T08:51:36.843000Z","leaderTransitions":37981}'
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	@VisibleForTesting
+	public static final String LEADER_ANNOTATION_KEY = "control-plane.alpha.kubernetes.io/leader";
+
+	private final ExecutorService executorService = Executors.newSingleThreadExecutor(
+		new ExecutorThreadFactory("KubernetesLeaderElector-ExecutorService"));
+
+	public KubernetesLeaderElector(
+			NamespacedKubernetesClient kubernetesClient,
+			String namespace,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderCallbackHandler leaderCallbackHandler) {
+		super(kubernetesClient, new LeaderElectionConfigBuilder()
+			.withName(leaderConfig.getConfigMapName())
+			.withLeaseDuration(leaderConfig.getLeaseDuration())
+			.withLock(new ConfigMapLock(namespace, leaderConfig.getConfigMapName(), leaderConfig.getLockIdentity()))
+			.withRenewDeadline(leaderConfig.getRenewDeadline())
+			.withRetryPeriod(leaderConfig.getRetryPeriod())
+			.withLeaderCallbacks(new LeaderCallbacks(
+				leaderCallbackHandler::isLeader,
+				leaderCallbackHandler::notLeader,
+				newLeader -> LOG.info("New leader elected {} for {}.", newLeader, leaderConfig.getConfigMapName())
+			))
+			.build());
+		LOG.info("Create KubernetesLeaderElector {} with lock identity {}.",
+			leaderConfig.getConfigMapName(), leaderConfig.getLockIdentity());
+	}
+
+	@Override
+	public void run() {
+		CompletableFuture.runAsync(super::run, executorService);

Review comment:
       We do not expect to have exception in the `LeaderElector#run`. Otherwise, it is fatal error handled by `FatalExitExceptionHandler`.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.DefaultKubeClientFactory;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubeClientFactory;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for the {@link KubernetesLeaderElector}. Start multiple leader contenders currently, one should elect
+ * successfully. And if current leader dies, a new one could take over.
+ */
+public class KubernetesLeaderElectorITCase {
+
+	@ClassRule
+	public static KubernetesResource kubernetesResource = new KubernetesResource();
+
+	private static final long TIMEOUT = 120L * 1000L;
+
+	private final KubeClientFactory kubeClientFactory = new DefaultKubeClientFactory();
+
+	private static final String LEADER_CONFIGMAP_NAME_PREFIX = "leader-test-cluster";
+
+	@Test
+	public void testMultipleKubernetesLeaderElectors() throws Exception {
+		final Configuration configuration = kubernetesResource.getConfiguration();
+		final ExecutorService executorService = kubernetesResource.getExecutorService();
+
+		final BlockingQueue<String> leaderStore = new LinkedBlockingQueue<>();
+		final String leaderConfigMapName = LEADER_CONFIGMAP_NAME_PREFIX + System.currentTimeMillis();
+		final int leaderNum = 3;
+
+		final KubernetesLeaderElector[] leaderElectors = new KubernetesLeaderElector[leaderNum];
+		// We use different Kubernetes clients for different leader electors.
+		final FlinkKubeClient[] kubeClients = new FlinkKubeClient[leaderNum];
+		final TestingLeaderCallbackHandler[] leaderCallbackHandlers = new TestingLeaderCallbackHandler[leaderNum];
+
+		try {
+			for (int i = 0; i < leaderNum; i++) {
+				kubeClients[i] = kubeClientFactory.fromConfiguration(configuration, executorService);
+				leaderCallbackHandlers[i] = new TestingLeaderCallbackHandler(leaderStore, UUID.randomUUID().toString());
+				final KubernetesLeaderElectionConfiguration leaderConfig = new KubernetesLeaderElectionConfiguration(
+					leaderConfigMapName, leaderCallbackHandlers[i].getLockIdentity(), configuration);
+				leaderElectors[i] = kubeClients[i].createLeaderElector(leaderConfig, leaderCallbackHandlers[i]);
+
+				// Start the leader electors to contend the leader
+				leaderElectors[i].run();
+			}
+
+			// Wait for the first leader
+			final String firstLockIdentity = leaderStore.poll(TIMEOUT, TimeUnit.MILLISECONDS);
+
+			for (int i = 0; i < leaderNum; i++) {
+				if (leaderCallbackHandlers[i].getLockIdentity().equals(firstLockIdentity)) {
+					// Check the callback is called.
+					assertThat(leaderCallbackHandlers[i].hasLeadership(), is(true));
+					// Current leader died
+					leaderElectors[i].stop();

Review comment:
       From the following piece of codes, we could find that `notLeader` will be called when the renew timeout of is interrupted. I will update the test to also verify the `notLeader` is called.
   
   ```
     /**
      * Starts the leader election loop
      */
     public void run() {
       LOGGER.debug("Leader election started");
       if (!acquire()) {
         return;
       }
       leaderElectionConfig.getLeaderCallbacks().onStartLeading();
       renewWithTimeout();
       leaderElectionConfig.getLeaderCallbacks().onStopLeading();
     }
   ```

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/TestingLeaderCallbackHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.kubeclient.resources;
+
+import java.util.concurrent.BlockingQueue;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Testing implementation for {@link KubernetesLeaderElector.LeaderCallbackHandler}.
+ */
+public class TestingLeaderCallbackHandler extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+	private final BlockingQueue<String> leaderStore;
+	private final String lockIdentity;
+	private boolean isLeader;
+
+	public TestingLeaderCallbackHandler(BlockingQueue<String> leaderStore, String lockIdentity) {
+		this.leaderStore = leaderStore;
+		this.lockIdentity = lockIdentity;
+	}
+
+	@Override
+	public void isLeader() {
+		isLeader = true;
+		leaderStore.poll();
+		leaderStore.offer(lockIdentity);
+		assertThat(leaderStore.size(), is(1));

Review comment:
       Make sense. I will make the queue private and add `waitForXXX` method.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final FlinkKubeClient kubeClient;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		checkNotNull(leaderConfig, "Leader election configuration");
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElector.run();
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}.", this);
+		leaderElector.stop();
+		kubernetesWatch.close();
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		assert(running);
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// Get the updated ConfigMap with new leader information
+						if (confirmedLeaderAddress == null) {
+							configMap.getData().remove(LEADER_ADDRESS_KEY);
+						} else {
+							configMap.getData().put(LEADER_ADDRESS_KEY, confirmedLeaderAddress);
+						}
+						if (confirmedLeaderSessionID == null) {
+							configMap.getData().remove(LEADER_SESSION_ID_KEY);
+						} else {
+							configMap.getData().put(LEADER_SESSION_ID_KEY, confirmedLeaderSessionID.toString());
+						}
+						configMap.getLabels().putAll(configMapLabels);
+						return Optional.of(configMap);
+					}
+					return Optional.empty();
+				}).get();
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(
+					"Successfully wrote leader information: Leader={}, session ID={}.",
+					confirmedLeaderAddress,
+					confirmedLeaderSessionID);
+			}
+		} catch (Exception e) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not write leader information since ConfigMap " + configMapName
+					+ " does not exist.", e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		assert(running);
+		final Optional<KubernetesConfigMap> configMapOpt = kubeClient.getConfigMap(configMapName);
+		if (configMapOpt.isPresent()) {
+			return KubernetesLeaderElector.hasLeadership(configMapOpt.get(), lockIdentity);
+		} else {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("ConfigMap " + configMapName + " does not exist.", null));
+			return false;
+		}
+	}
+
+	private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+		@Override
+		public void isLeader() {
+			leaderElectionEventHandler.onGrantLeadership();
+		}
+
+		@Override
+		public void notLeader() {
+			leaderElectionEventHandler.onRevokeLeadership();
+			// Continue to contend the leader
+			leaderElector.run();
+		}
+	}
+
+	private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> {
+		@Override
+		public void onAdded(List<KubernetesConfigMap> configMaps) {
+			// noop
+		}
+
+		@Override
+		public void onModified(List<KubernetesConfigMap> configMaps) {
+			// We should only receive events for the watched ConfigMap
+			final KubernetesConfigMap configMap = checkConfigMaps(configMaps, configMapName);
+
+			if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+				leaderElectionEventHandler.onLeaderInformationChange(getLeaderInformationFromConfigMap(configMap));

Review comment:
       Copy what I have added in the description of `LeaderElectionEventHandler#onLeaderInformationChange`. Does it make sense? Or you still believe that we should keep a local cache in `KubernetesLeaderElectionDriver` to check whether the leader information is truly changed.
   
   ```
   Duplicated leader change events could happen, so the implementation should check whether the passed leader information is really different with internal confirmed leader information.
   ```

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.runtime.leaderelection.LeaderInformation;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_KEY;
+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.hamcrest.Matchers.containsString;
+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 KubernetesLeaderElectionDriver}.
+ */
+public class KubernetesLeaderElectionDriverTest extends KubernetesHighAvailabilityTestBase {
+
+	@Test
+	public void testIsLeader() throws Exception {
+		new Context() {{
+			runTest(
+				() -> {
+					// Grant leadership
+					leaderCallbackGrantLeadership();
+					assertThat(electionEventHandler.isLeader(), is(true));
+					assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LEADER_INFORMATION));
+				});
+		}};
+	}
+
+	@Test
+	public void testNotLeader() throws Exception {
+		new Context() {{
+			runTest(
+				() -> {
+					leaderCallbackGrantLeadership();
+					// Revoke leadership
+					getLeaderCallback().notLeader();
+
+					electionEventHandler.waitForRevokeLeader(TIMEOUT);
+					assertThat(electionEventHandler.isLeader(), is(false));
+					assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LeaderInformation.empty()));
+					// The ConfigMap should also be cleared
+					assertThat(getLeaderConfigMap().getData().get(LEADER_ADDRESS_KEY), is(nullValue()));
+					assertThat(getLeaderConfigMap().getData().get(LEADER_SESSION_ID_KEY), is(nullValue()));

Review comment:
       Because we do not have new leader elected and the lock identity annotation on the ConfigMap is still the old one. So the `KubernetesLeaderElector#hasLeadership` could pass the check. I think this is the expected behavior. Once the leader ConfigMap is over written by a new leader. It should return false.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory);
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		synchronized (lock) {
+			leaderContender = contender;
+			leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver(
+				this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription());
+			LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping DefaultLeaderElectionService.");
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.close();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+							"LeaderElectionService has already been stopped.", leaderSessionID);
+					}
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+							"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+					}
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			if (running) {
+				return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("hasLeadership is called after the service is stopped, returning false.");
+				}
+				return false;
+			}
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+		leaderElectionDriver.writeLeaderInformation(
+			LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress));
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+
+				leaderContender.grantLeadership(issuedLeaderSessionID);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Ignoring the grant leadership notification since the {} has " +
+						"already been closed.", leaderElectionDriver);
+				}
+			}
+		}
+	}
+
+	@Override
+	@GuardedBy("lock")
+	public void onRevokeLeadership() {
+		synchronized (lock) {
+			if (running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+				}
+
+				issuedLeaderSessionID = null;
+				clearConfirmedLeaderInformation();
+
+				leaderContender.revokeLeadership();
+
+				// Clear the old leader information on the external storage
+				leaderElectionDriver.writeLeaderInformation(LeaderInformation.empty());

Review comment:
       The reason why I want to do this is to process the following situation.
   * Old leader lost leadership and new leader is not launched very fast(maybe 120s or longer).
   * Old leader should clear the leader information on ConfigMap so that the listener could know this event.
   
   For Kubernetes, it could work. Because no new leader has been elected and the lock identity annotation is still the old one.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509281041



##########
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);

Review comment:
       Yes, no exception should be thrown here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r506226269



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -258,6 +258,13 @@
 			.withDescription("If configured, Flink will add \"resources.limits.<config-key>\" and \"resources.requests.<config-key>\" " +
 				"to the main container of TaskExecutor and set the value to the value of " + ExternalResourceOptions.EXTERNAL_RESOURCE_AMOUNT.key() + ".");
 
+	public static final ConfigOption<Integer> KUBERNETES_MAX_RETRY_ATTEMPTS =
+		key("kubernetes.client.max-retry-attempts")
+			.intType()
+			.defaultValue(5)
+			.withDescription("Defines the number of Kubernetes resources update operation retries before the client " +
+				"gives up. For example, updating the ConfigMap.");
+

Review comment:
       Currently, the retry attempt only applied to the `checkAndUpdate` transactional operation. So the new name makes sense. Will update it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513920680



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}", this);
+
+			client.getUnhandledErrorListenable().removeListener(this);
+
+			client.getConnectionStateListenable().removeListener(listener);
+
+			Exception exception = null;
+
+			try {
+				cache.close();
+			} catch (Exception e) {
+				exception = e;
+			}
+
+			try {
+				leaderLatch.close();
+			} catch (Exception e) {
+				exception = ExceptionUtils.firstOrSuppressed(e, exception);
+			}
+
+			if (exception != null) {
+				throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		synchronized (lock) {
+			if (running) {
+				leaderElectionEventHandler.onGrantLeadership();
+			}
+		}

Review comment:
       We could use the `lock` in DefaultLeaderElectionService to easily filter the spurious callbacks. When the `leaderElectionDriver` is closed, the `running` is also false. Then we check the `running` in the callback implementation methods(e.g. `onGrantLeadership`, `onRevokeLeadership`, `onLeaderInformationChange`).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8720",
       "triggerID" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753",
       "triggerID" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884",
       "triggerID" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8956",
       "triggerID" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8988",
       "triggerID" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce1452d50dcad0cd1e6decb42e26c3c352ee8d67",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ce1452d50dcad0cd1e6decb42e26c3c352ee8d67",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * e2795d9209fa2241cf0909737df9dca17defea4e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8988) 
   * ce1452d50dcad0cd1e6decb42e26c3c352ee8d67 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509856352



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						}
+						return updated;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),

Review comment:
       What I mean is the ConfigMap could be created in the `KubernetesLeaderElectionService#Watcher`. So even the first we get a `Optional.empty()`, we could a get correct ConfigMap by retrying.
   
   I will add a two tests here.
   * ConfigMap always does not exists and retry failed
   * `createOrReplace` failed and should retry




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507555036



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();

Review comment:
       Because we are not completely sure whether start the necessary services(e.g. `LeaderLatch`, `NodeCache`, `KubernetesLeaderElector`) is thread safe. It is a easy way to always put them in the `synchronized` block. I will keep them guarded by this lock if you do not have other concerns.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513904491



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}", this);
+
+			client.getUnhandledErrorListenable().removeListener(this);
+
+			client.getConnectionStateListenable().removeListener(listener);
+
+			Exception exception = null;
+
+			try {
+				cache.close();
+			} catch (Exception e) {
+				exception = e;
+			}
+
+			try {
+				leaderLatch.close();
+			} catch (Exception e) {
+				exception = ExceptionUtils.firstOrSuppressed(e, exception);
+			}
+
+			if (exception != null) {
+				throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		synchronized (lock) {
+			if (running) {
+				leaderElectionEventHandler.onGrantLeadership();
+			}
+		}

Review comment:
       I have noticed the deadlock problem yesterday and moved `leaderElectionDriver.close()` out of the outer lock. This will solve the deadlock problem now. But it is fragile and not very easy to understand. Maybe in the future we introduce some other interfaces, which will be called in the outer lock, then we still have the potential deadlock.
   
   Not execute the callback under the inner lock could completely avoid the deadlock. Just like you say, the implementation(e.g. `DefaultLeaderElection`) needs to filter out the spurious callbacks. This is easier to understand.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507479524



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -104,6 +107,67 @@ KubernetesWatch watchPodsAndDoCallback(
 		Map<String, String> labels,
 		WatchCallbackHandler<KubernetesPod> podCallbackHandler);
 
+	/**
+	 * Create the ConfigMap with specified content. If the ConfigMap already exists, nothing will happen.
+	 *
+	 * @param configMap ConfigMap.
+	 *
+	 * @return Return the ConfigMap create future.
+	 */
+	CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap);

Review comment:
       After a consideration, I will change the behavior of `CreateConfigMap`. If it is already existing, a `FlinkRuntimeExeception` will be thrown. Because the existing ConfigMap may be created externally and will make the Flink cluster works not as expected.
   
   So the name will still be `createConfigMap`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507614859



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServicesFactory.java
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.kubeclient.KubeClientFactory;
+import org.apache.flink.runtime.blob.BlobUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesFactory;
+
+import java.util.concurrent.Executor;
+
+/**
+ * Factory for creating Kubernetes high availability services.
+ */
+public class KubernetesHaServicesFactory implements HighAvailabilityServicesFactory {
+
+	@Override
+	public HighAvailabilityServices createHAServices(Configuration configuration, Executor executor) throws Exception {
+		return new KubernetesHaServices(
+			KubeClientFactory.fromConfiguration(configuration),

Review comment:
       Since the `Fabric8FlinkKubeClient` is thread safe. I will return a singleton in the `KubeClientFactory`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509220770



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	protected volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	protected volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	protected volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			internalStop();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (logger.isDebugEnabled()) {
+			logger.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		if (hasLeadership()) {
+			// check if this is an old confirmation call
+			synchronized (lock) {
+				if (running) {
+					if (leaderSessionID.equals(this.issuedLeaderSessionID)) {
+						confirmLeaderInformation(leaderSessionID, leaderAddress);
+						writeLeaderInformation();
+					}
+				} else {
+					logger.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"LeaderElectionService has already been stopped.", leaderSessionID);
+				}
+			}
+		} else {
+			logger.warn("The leader session ID {} was confirmed even though the " +
+				"corresponding JobManager was not elected as the leader.", leaderSessionID);
+		}
+	}
+
+	@Override
+	public final boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		return hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);

Review comment:
       I think it is the old behavior. We do not put this piece of codes in the `lock`. I will make this method guarded by `lock` in the refactor.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507620149



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionService.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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 java.util.List;
+import java.util.Map;
+import java.util.Objects;
+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.watchConfigMapsAndDoCallback(configMapName, new ConfigMapCallbackHandlerImpl());
+	}
+
+	@Override
+	public void internalStop() {
+		if (kubernetesWatch != null) {
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	protected void writeLeaderInformation() {
+		updateConfigMap(configMapName);
+	}
+
+	@Override
+	protected boolean checkLeaderLatch() {
+		return kubeClient.getConfigMap(configMapName)
+			.map(configMap -> KubernetesUtils.getLeaderChecker().test(configMap))
+			.orElse(false);
+	}
+
+	@Override
+	public String toString() {
+		return "KubernetesLeaderElectionService{configMapName='" + configMapName + "'}";
+	}
+
+	private void updateConfigMap(String configMapName) {

Review comment:
       Make sense.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508993402



##########
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:
       Just like Yarn MiniCluster, we are not very simple to start a minikube. So I prefer to add the unit tests for the contract testing first and leave the integration tests with E2E tests together. WDYT?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507515399



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	protected volatile LeaderContender leaderContender;
+
+	private volatile UUID issuedLeaderSessionID;
+
+	protected volatile UUID confirmedLeaderSessionID;
+
+	protected volatile String confirmedLeaderAddress;
+
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+		}
+
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		internalStop();
+	}

Review comment:
       Actually, there's no valid harm to also put `internalStop` in the `synchronized` block. Since the old behavior in `ZookeeperLeaderElectionService` is like this, I keep them in the `AbstractLeaderElectionService`. Just like you said, it is a little hard for the maintenance. I agree that we could move the `internalStop` into the `synchronized` block.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513355235



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.leaderelection;
+
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderContender} and
+ * {@link LeaderElectionEventHandler}.
+ */
+public class TestingLeaderBase {
+
+	protected boolean leader = false;
+	protected Throwable error = null;
+
+	protected final Object lock = new Object();
+	private final Object errorLock = new Object();
+
+	/**
+	 * Waits until the contender becomes the leader or until the timeout has been exceeded.
+	 *
+	 * @param timeout
+	 * @throws TimeoutException
+	 */
+	public void waitForLeader(long timeout) throws TimeoutException {
+		waitFor(this::isLeader, timeout, "Contender was not elected as the leader within " + timeout + "ms");
+	}
+
+	/**
+	 * Waits until the contender revokes the leader or until the timeout has been exceeded.
+	 *
+	 * @param timeout
+	 * @throws TimeoutException
+	 */
+	public void waitForRevokeLeader(long timeout) throws TimeoutException {
+		waitFor(() -> !isLeader(), timeout, "Contender was not revoked within " + timeout + "ms");
+	}
+
+	protected void waitFor(Supplier<Boolean> supplier, long timeout, String msg) throws TimeoutException {
+		long start = System.currentTimeMillis();
+		long curTimeout;
+
+		while (!supplier.get() && (curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
+			synchronized (lock) {
+				try {
+					lock.wait(curTimeout);
+				} catch (InterruptedException e) {
+					// we got interrupted so check again for the condition
+				}
+			}
+		}
+
+		if (!supplier.get()) {
+			throw new TimeoutException(msg);
+		}

Review comment:
       This is not strictly needed right now. We could tackle this as part of a follow up issue.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderRetrievalListener} and
+ * {@link org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler}.
+ */
+public class TestingRetrievalBase {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+	private final Object lock = new Object();
+
+	private String address;
+	private String oldAddress;
+	private UUID leaderSessionID;
+	private Exception exception;
+
+
+	public String getAddress() {
+		return address;
+	}
+
+	public UUID getLeaderSessionID() {
+		return leaderSessionID;
+	}
+
+	public String waitForNewLeader(long timeout) throws Exception {
+		long start = System.currentTimeMillis();
+		long curTimeout;
+
+		synchronized (lock) {
+			while (
+				exception == null &&
+					(address == null || address.equals(oldAddress)) &&
+					(curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
+				try {
+					lock.wait(curTimeout);
+				} catch (InterruptedException e) {
+					// we got interrupted so check again for the condition
+				}
+			}
+		}

Review comment:
       Same here. This can be a follow up task.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509292861



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {

Review comment:
       If you mean testing the real `KubernetesLeaderElector` behavior, I am afraid we could not do it in the UT. Because we have no minikube cluster and the `KubernetesMockServer` do not support the leader election.
   
   If you mean to add a test for `KubernetesLeaderElector#hasLeadership`, I will do it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509043134



##########
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:
       I see. Hmm, thinking about it again I came to the conclusion that such a safety mechanism against accidental deletions might harden operations. I wouldn't have said that we need it for the first version but since you have it already implemented, let's keep it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509240356



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	/** The executor to run external IO operations on. */
+	protected final Executor executor;
+
+	/** The runtime configuration. */
+	protected final Configuration configuration;
+
+	/** Store for arbitrary blobs. */
+	private final BlobStoreService blobStoreService;
+
+	public AbstractHaServices(Executor executor, Configuration config, BlobStoreService blobStoreService) {
+
+		this.executor = checkNotNull(executor);
+		this.configuration = checkNotNull(config);
+		this.blobStoreService = blobStoreService;
+	}
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForResourceManager());
+	}
+
+	@Override
+	public LeaderRetrievalService getDispatcherLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForDispatcher());
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		return createLeaderRetrievalService(getLeaderNameForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress) {
+		return getJobManagerLeaderRetriever(jobID);
+	}
+
+	@Override
+	public LeaderRetrievalService getClusterRestEndpointLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForRestServer());
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForResourceManager());
+	}
+
+	@Override
+	public LeaderElectionService getDispatcherLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForDispatcher());
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		return createLeaderElectionService(getLeaderNameForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderElectionService getClusterRestEndpointLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForRestServer());
+	}
+
+	@Override
+	public BlobStore createBlobStore() {
+		return blobStoreService;
+	}
+
+	@Override
+	public void close() throws Exception {
+		Throwable exception = null;
+
+		try {
+			blobStoreService.close();
+		} catch (Throwable t) {
+			exception = t;
+		}
+
+		internalClose();
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(exception, "Could not properly close the " + getClass().getSimpleName());
+		}
+	}
+
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		logger.info("Close and clean up all data for {}.", getClass().getSimpleName());
+
+		Throwable exception = null;
+
+		try {
+			blobStoreService.closeAndCleanupAllData();
+		} catch (Throwable t) {
+			exception = t;
+		}
+
+		try {
+			internalCleanup();
+		} catch (Throwable t) {
+			exception = ExceptionUtils.firstOrSuppressed(t, exception);
+		}
+
+		internalClose();
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(
+				exception, "Could not properly close and clean up all data of high availability service.");
+		}
+		logger.info("Finished cleaning up the high availability data.");
+	}
+
+	/**
+	 * Create leader election service with specified leaderName.
+	 * @param leaderName ConfigMap name in Kubernetes or child node path in Zookeeper.
+	 *
+	 * @return Return LeaderElectionService using Zookeeper or Kubernetes.
+	 */
+	protected abstract LeaderElectionService createLeaderElectionService(String leaderName);
+
+	/**
+	 * Create leader retrieval service with specified leaderName.
+	 * @param leaderName ConfigMap name in Kubernetes or child node path in Zookeeper.
+	 *
+	 * @return Return LeaderRetrievalService using Zookeeper or Kubernetes.
+	 */
+	protected abstract LeaderRetrievalService createLeaderRetrievalService(String leaderName);
+
+	/**
+	 * Closes components which don't distinguish between close and closeAndCleanupAllData.
+	 */
+	protected abstract void internalClose() throws Exception;
+
+	/**
+	 * Clean up the meta data in the distributed system(e.g. Zookeeper, Kubernetes ConfigMap).
+	 */
+	protected abstract void internalCleanup() throws Exception;

Review comment:
       I will add a comment here.
   
   ```
   	 * <p>If an exception occurs during internal cleanup, we will continue the cleanup in
   	 * {@link #closeAndCleanupAllData} and report exceptions only after all cleanup steps
   	 * have been attempted.
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507421985



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -103,6 +121,52 @@ public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, WatchC
 		return watchPodsAndDoCallbackFunction.apply(labels, podCallbackHandler);
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		configMapStore.putIfAbsent(configMap.getName(), configMap);
+		return CompletableFuture.completedFuture(null);
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final KubernetesConfigMap configMap = configMapStore.get(name);
+		if (configMap == null) {
+			return Optional.empty();
+		}
+		return Optional.of(new MockKubernetesConfigMap(configMap.getName(), new HashMap<>(configMap.getData())));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			Predicate<KubernetesConfigMap> checker,
+			FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function) {
+		return getConfigMap(configMapName).map(FunctionUtils.uncheckedFunction(
+			configMap -> {
+				final boolean shouldUpdate = checker.test(configMap);
+				if (shouldUpdate) {
+					configMapStore.put(configMap.getName(), function.apply(configMap));
+				}
+				return CompletableFuture.completedFuture(shouldUpdate);
+			}))
+			.orElseThrow(() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists."));
+	}

Review comment:
       Since all the new introduced operations are ConfigMap related. So I add a new field `configMapStore` to manage the state. It is more natural to create/update/read/delete the ConfigMap state.
   
   Just like you say, if we want to delete a ConfigMap externally. We just need to call `configMapStore#remove(name)`. And the `configMapStore` in `TestingFlinkKubeClient#Builder` could be passed externally.
   
   If you insist, I could add several `Function`s instead.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509945969



##########
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:
       Shouldn't these tests be part of the commits where we add the K8s services? That way it will be a bit easier to review them for completeness.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508355344



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java
##########
@@ -80,4 +84,14 @@
 	public static final String RESTART_POLICY_OF_NEVER = "Never";
 
 	public static final String NATIVE_KUBERNETES_COMMAND = "native-k8s";
+
+	// Constants for Kubernetes high availability
+	public static final String LEADER_ADDRESS_KEY = "address";
+	public static final String LEADER_SESSION_ID_KEY = "sessionId";
+	public static final String CHECKPOINT_COUNTER_KEY = "counter";
+	public static final String RUNNING_JOBS_REGISTRY_KEY_PREFIX = "runningJobsRegistry";
+	public static final String JOB_GRAPH_STORE_KEY_PREFIX = "jobGraph";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();

Review comment:
       I'm not sure about sharing the identity across leader election services. By making `LOCK_IDENTITY` static, we implicitly introduces a constraint that we cannot have multiple contenders in the same process that competing for leadership of the same service. This may not be a problem for Flink's current implementation, but I would suggest to not making such assumption if it doesn't add too much effort.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509937977



##########
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:
       Would we do these kind of calls in the update function of the configMap?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509276361



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,69 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateConfigMap() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+		final Optional<KubernetesConfigMap> currentOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(currentOpt.isPresent(), is(true));
+		assertThat(currentOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	@Test
+	public void testDeleteConfigMapByLabels() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(TESTING_LABELS);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testDeleteConfigMapByName() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(
+			TESTING_CONFIG_MAP_NAME,
+			c -> {
+				c.getData().put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_NEW_VALUE);
+				return Optional.of(c);
+			}).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_NEW_VALUE));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMapWithEmptyResult() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker not pass and return empty result
+		this.flinkKubeClient.checkAndUpdateConfigMap(TESTING_CONFIG_MAP_NAME, c -> Optional.empty()).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	private KubernetesConfigMap buildTestingConfigMap() {
+		final Map<String, String> data = new HashMap<>();
+		data.put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_VALUE);
+		return new KubernetesConfigMap(new ConfigMapBuilder()
+			.withNewMetadata()
+			.withName(TESTING_CONFIG_MAP_NAME)
+			.withLabels(TESTING_LABELS)
+			.endMetadata()
+			.withData(data).build());
+	}

Review comment:
       The reason why I do not test the concurrent writes is that we are using `KubernetesMockServer`. Actually, it could not support such semantic, the resource version is always null. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508995602



##########
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) {

Review comment:
       If we agree to not handle the externally deletion/update, then the `Watcher` could be completely removed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509244080



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {

Review comment:
       Hmm. For `LeaderElectionService`, I agree with you that the `lock` should be dealt very carefully and it is not very easy to do in the sub classes. So I will factor the `LeaderElectionService` with composition.
   
   However, for the `AbstractHaServices`, we do not run into this situation and the subclass is very easy to implement. Another concern is lots of codes will be touched if we rename the `ZookeeperHaServices` to `ZookeeperHaDriver` or sth similar. I have tried and find it is not worth.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509282229



##########
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) {

Review comment:
       I will fix this in the composition refactor.

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

Review comment:
       I will fix this in the composition refactor.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8720",
       "triggerID" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753",
       "triggerID" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884",
       "triggerID" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8956",
       "triggerID" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8988",
       "triggerID" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce1452d50dcad0cd1e6decb42e26c3c352ee8d67",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9013",
       "triggerID" : "ce1452d50dcad0cd1e6decb42e26c3c352ee8d67",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * e2795d9209fa2241cf0909737df9dca17defea4e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8988) 
   * ce1452d50dcad0cd1e6decb42e26c3c352ee8d67 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9013) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509944166



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {

Review comment:
       Hmm, I actually believe that we should have some form of testing of the Kubernetes HA services against a real/testing K8s server. I mean it can run as part of the E2E tests but I wouldn't solely rely on running some Flink jobs using these services. With this approach it will be super hard to cover more complicated scenarios.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508993402



##########
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:
       Unlike Yarn MiniCluster, we are not very simple to start a minikube. So I prefer to add the unit tests for the contract testing first and leave the integration tests with E2E tests together. WDYT?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507402405



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,58 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateAndDeleteConfigMap() {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(haLabels);
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(false));
+	}

Review comment:
       Will separate them into different test cases.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508509211



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	protected static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	protected static final String LEADER_ANNOTATION_KEY = "control-plane.alpha.kubernetes.io/leader";

Review comment:
       For `LOCK_IDENTITY`, I will factor it out. And make it could specified via `KubernetesLeaderElectionConfiguration`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708937060


   cc @xintongsong @tillrohrmann Could you please have a look at your convenience?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509017125



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -103,6 +121,52 @@ public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, WatchC
 		return watchPodsAndDoCallbackFunction.apply(labels, podCallbackHandler);
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		configMapStore.putIfAbsent(configMap.getName(), configMap);
+		return CompletableFuture.completedFuture(null);
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final KubernetesConfigMap configMap = configMapStore.get(name);
+		if (configMap == null) {
+			return Optional.empty();
+		}
+		return Optional.of(new MockKubernetesConfigMap(configMap.getName(), new HashMap<>(configMap.getData())));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			Predicate<KubernetesConfigMap> checker,
+			FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function) {
+		return getConfigMap(configMapName).map(FunctionUtils.uncheckedFunction(
+			configMap -> {
+				final boolean shouldUpdate = checker.test(configMap);
+				if (shouldUpdate) {
+					configMapStore.put(configMap.getName(), function.apply(configMap));
+				}
+				return CompletableFuture.completedFuture(shouldUpdate);
+			}))
+			.orElseThrow(() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists."));
+	}

Review comment:
       Hmm. I know your concerns now. Then I will keep the `configMapStore` in `TestingFlinkKubeClient` internally. And add `Functions` for the flexibility, for example, throw an exception when `createConfigMap`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513218321



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.leaderelection;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		final TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory testingLeaderElectionDriverFactory =
+			new TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory();
+		final DefaultLeaderElectionService leaderElectionService = new DefaultLeaderElectionService(
+			testingLeaderElectionDriverFactory);
+		final TestingContender testingContender = new TestingContender(TEST_URL, leaderElectionService);
+		leaderElectionService.start(testingContender);
+
+		// grant leadership
+		final TestingLeaderElectionDriver testingLeaderElectionDriver =
+			testingLeaderElectionDriverFactory.getCurrentLeaderDriver();
+		assertThat(testingLeaderElectionDriver, is(notNullValue()));
+		testingLeaderElectionDriver.isLeader();
+
+		testingContender.waitForLeader(timeout);
+		assertThat(testingContender.isLeader(), is(true));
+		assertThat(testingContender.getDescription(), is(TEST_URL));
+
+		// Check the external storage
+		assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+
+		// revoke leadership
+		testingLeaderElectionDriver.notLeader();
+		testingContender.waitForRevokeLeader(timeout);
+		assertThat(testingContender.isLeader(), is(false));
+
+		leaderElectionService.stop();
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		final TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory testingLeaderElectionDriverFactory =
+			new TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory();
+		final DefaultLeaderElectionService leaderElectionService = new DefaultLeaderElectionService(
+			testingLeaderElectionDriverFactory);
+		final TestingContender testingContender = new TestingContender(TEST_URL, leaderElectionService);
+		leaderElectionService.start(testingContender);
+
+		final TestingLeaderElectionDriver testingLeaderElectionDriver =
+			testingLeaderElectionDriverFactory.getCurrentLeaderDriver();
+		assertThat(testingLeaderElectionDriver, is(notNullValue()));
+		testingLeaderElectionDriver.isLeader();
+		testingContender.waitForLeader(timeout);
+
+		// Leader information changed and should be corrected
+		testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+		assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));

Review comment:
       Will check it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508986718



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubeClientFactory.java
##########
@@ -71,7 +73,12 @@ public static FlinkKubeClient fromConfiguration(Configuration flinkConfig) {
 
 		final KubernetesClient client = new DefaultKubernetesClient(config);
 
-		return new Fabric8FlinkKubeClient(flinkConfig, client, KubeClientFactory::createThreadPoolForAsyncIO);
+		if (flinkKubeClient == null) {
+			flinkKubeClient = new Fabric8FlinkKubeClient(
+				flinkConfig, client, KubeClientFactory::createThreadPoolForAsyncIO);
+		}

Review comment:
       Currently, in the JobManager process, we have two `flinkKubeClient`s. One is for `KubernetesResourceDriver`, and the other is for `KubernetesHaService`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508994167



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						}
+						return updated;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),

Review comment:
       If we agree to not handle the externally deletion/update, then we do not need to retry if the ConfigMap not exists.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508989034



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+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.NAME_SEPARATOR;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * High availability service for Kubernetes.
+ */
+public class KubernetesHaServices extends AbstractHaServices {
+
+	private final String clusterId;
+
+	/** Kubernetes client. */
+	private final FlinkKubeClient kubeClient;
+
+	private static final String RESOURCE_MANAGER_NAME = "resourcemanager";
+
+	private static final String DISPATCHER_NAME = "dispatcher";
+
+	private static final String JOB_MANAGER_NAME = "jobmanager";
+
+	private static final String REST_SERVER_NAME = "restserver";
+
+	private static final String LEADER_SUFFIX = "leader";
+
+	KubernetesHaServices(
+			FlinkKubeClient kubeClient,
+			Executor executor,
+			Configuration config,
+			BlobStoreService blobStoreService) {
+
+		super(executor, config, blobStoreService);
+		this.kubeClient = checkNotNull(kubeClient);
+		this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID));
+	}
+
+	@Override
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
+		return new StandaloneCheckpointRecoveryFactory();
+	}
+
+	@Override
+	public JobGraphStore getJobGraphStore() {
+		return new StandaloneJobGraphStore();
+	}
+
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() {
+		return new StandaloneRunningJobsRegistry();
+	}
+
+	@Override
+	public LeaderElectionService createLeaderElectionService(String leaderName) {
+		return new KubernetesLeaderElectionService(
+			kubeClient,
+			executor,
+			KubernetesLeaderElectionConfiguration.fromConfiguration(leaderName, configuration));
+	}
+
+	@Override
+	public LeaderRetrievalService createLeaderRetrievalService(String leaderName) {
+		return new KubernetesLeaderRetrievalService(kubeClient, leaderName);
+	}
+
+	@Override
+	public void internalClose() throws Exception {
+		kubeClient.close();

Review comment:
       Yes. You are right. So I will keep two separate `flinkKubeClient` for `KubernetesHaService` and `KubernetesResourceDriver`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-716414850


   @tillrohrmann I have address the comments you left and pushed again. Compared with previous PR, you could find the following changes.
   
   * Introduce `LeaderElectionEventHandler` and `LeaderRetrievalEventHandler`. They define the contracts between `LeaderElectionDriver`/`LeaderRetrievalDriver` and `DefaultLeaderElectionService`/`DefaultLeaderRetrievalService`.
   * Make some `ZooKeeperLeaderElection` tests could directly work with driver
   * Add test for `DefaultLeaderElectionService`
   * Use driver(not instantiate `DefaultLeaderElectionService`) for the HA unit tests. Then the tests are more purposeful.
   * Add ITCases for Kubernetes complicated test case, concurrent leader election, concurrent `checkAndUpdateConfigMap`


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 991bdfa5427479f845abaf91caba0069011ee9e3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507479524



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -104,6 +107,67 @@ KubernetesWatch watchPodsAndDoCallback(
 		Map<String, String> labels,
 		WatchCallbackHandler<KubernetesPod> podCallbackHandler);
 
+	/**
+	 * Create the ConfigMap with specified content. If the ConfigMap already exists, nothing will happen.
+	 *
+	 * @param configMap ConfigMap.
+	 *
+	 * @return Return the ConfigMap create future.
+	 */
+	CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap);

Review comment:
       After a serious consideration, I decide to change the behavior of `CreateConfigMap`. If it is already existing, a `FlinkRuntimeExeception` will be thrown. Because the existing ConfigMap may be created externally wrongly and this will make the Flink cluster works not as expected. Moreover, only the leader could create the ConfigMap. We will not run into the concurrent modification.
   
   So the name will still be `createConfigMap`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509057766



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						}
+						return updated;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),

Review comment:
       I think this is independent of this question because `checkAndUpdateConfigMap` should never create a non-existing config map according to the code. If `getConfigMap()` returns `Optional.empty()` then we will throw the `FlinkRuntimeException`, retry and end up again receiving an `Optional.empty()`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann closed pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann closed pull request #13644:
URL: https://github.com/apache/flink/pull/13644


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514879987



##########
File path: flink-end-to-end-tests/test-scripts/test_kubernetes_itcases.sh
##########
@@ -0,0 +1,30 @@
+#!/usr/bin/env bash
+################################################################################
+# 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.
+################################################################################
+
+source "$(dirname "$0")"/common_kubernetes.sh
+
+start_kubernetes
+
+# Set the ITCASE_KUBECONFIG environment since it is required to run the ITCases
+export ITCASE_KUBECONFIG=~/.kube/config
+
+cd $END_TO_END_DIR/../flink-kubernetes

Review comment:
       Yes. It will make the ITCase run longer as the Kubernetes ITCases get more and more. Currently, I will try to keep the script here. Actually, if you want to debug the ITCase in your IDE or run in your local machine with the `test_kubernetes_itcases.sh`. Both of them could work.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509847440



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	protected volatile LeaderContender leaderContender;
+
+	private volatile UUID issuedLeaderSessionID;
+
+	protected volatile UUID confirmedLeaderSessionID;
+
+	protected volatile String confirmedLeaderAddress;
+
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+		}
+
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		internalStop();
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (logger.isDebugEnabled()) {
+			logger.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		if (checkLeaderLatch()) {

Review comment:
       This will be done in the composition refactor.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r506222695



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesWatcher.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.client.KubernetesClientException;
+import io.fabric8.kubernetes.client.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Watcher for resources in Kubernetes.
+ */
+public abstract class KubernetesWatcher<T extends HasMetadata, K extends KubernetesResource<T>> implements Watcher<T> {

Review comment:
       Make sense. Will move to a separate commit.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e33c52394024e8f1312c096a4b612d53d8848b97 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514074824



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_KEY;
+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.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link DefaultLeaderElectionService} with {@link KubernetesLeaderElectionDriver}.
+ */
+public class KubernetesLeaderElectionTest extends KubernetesHighAvailabilityTestBase {
+
+	@Test
+	public void testKubernetesLeaderElection() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					assertThat(electionEventHandler.getLeaderInformation().getLeaderAddress(), is(LEADER_URL));
+
+					// Revoke leader
+					leaderController.set(false);
+					electionEventHandler.waitForRevokeLeader(TIMEOUT);
+					assertThat(electionEventHandler.getLeaderInformation(), is(LeaderInformation.empty()));
+				});
+		}};

Review comment:
       Actually, this test is for leader information should cleared after revoke. I will make the test content more clear.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-713591200


   @tillrohrmann I have gone though all your comments and will update the PR soon. Just two quick points need to confirm with you.
   
   * Do you think we should not handle the externally deletion/update? For example, a Flink cluster with HA configured is running, some user delete/update the ConfigMap via `kubectl`. If it is yes, I will remove the `Watcher` in `KubernetesLeaderElectionService`. And change some "ConfigMap not exists" behavior.
   * I am afraid it is hard to use a real K8s server in the UT because it is not very easy to start a `minikube`. I will try to add the unit tests for the contract testing now and leave the real cluster test in E2E test implementation. Does it make sense?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r510031269



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {

Review comment:
       This is an interesting solution. Unlike `LeaderElectionService`, we could not get too much benefit from this. So I will  leave this refactor in the future when we want to touch the `ZookeeperHaService` again. For example, get rid of the `lock-and-release` and use a K8s similar transactional operation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507402293



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,58 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateAndDeleteConfigMap() {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(haLabels);
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+
+		final Supplier<Exception> configMapNotExistException = () -> new Exception("ConfigMap not exist");
+		FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function = c -> {
+			c.getData().put(LEADER_ADDRESS_KEY, LEADER_ADDRESS_NEW);
+			return c;
+		};
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+
+		// Checker not pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(LEADER_CONFIG_MAP_NAME, c -> false, function).get();
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(LEADER_CONFIG_MAP_NAME, c -> true, function).get();
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS_NEW));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+	}
+
+	private KubernetesConfigMap buildHAConfigMap() {
+		final Map<String, String> data = new HashMap<>();
+		data.put(LEADER_ADDRESS_KEY, LEADER_ADDRESS);
+		return new KubernetesConfigMap(new ConfigMapBuilder()
+			.withNewMetadata()
+			.withName(LEADER_CONFIG_MAP_NAME)
+			.withLabels(haLabels)
+			.endMetadata()
+			.withData(data).build());
+	}

Review comment:
       Will update.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,58 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateAndDeleteConfigMap() {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(haLabels);
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(false));
+	}

Review comment:
       Will update.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508490184



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubeClientFactory.java
##########
@@ -71,7 +73,12 @@ public static FlinkKubeClient fromConfiguration(Configuration flinkConfig) {
 
 		final KubernetesClient client = new DefaultKubernetesClient(config);
 
-		return new Fabric8FlinkKubeClient(flinkConfig, client, KubeClientFactory::createThreadPoolForAsyncIO);
+		if (flinkKubeClient == null) {
+			flinkKubeClient = new Fabric8FlinkKubeClient(
+				flinkConfig, client, KubeClientFactory::createThreadPoolForAsyncIO);
+		}

Review comment:
       I second your idea and will revert this commit. Then we could leave the shared `flinkKubeClient` as a follow-up optimization.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r505312805



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesWatcher.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.client.KubernetesClientException;
+import io.fabric8.kubernetes.client.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Watcher for resources in Kubernetes.
+ */
+public abstract class KubernetesWatcher<T extends HasMetadata, K extends KubernetesResource<T>> implements Watcher<T> {

Review comment:
       Let's rename this class to `AbstractKubernetesWatcher`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -258,6 +258,13 @@
 			.withDescription("If configured, Flink will add \"resources.limits.<config-key>\" and \"resources.requests.<config-key>\" " +
 				"to the main container of TaskExecutor and set the value to the value of " + ExternalResourceOptions.EXTERNAL_RESOURCE_AMOUNT.key() + ".");
 
+	public static final ConfigOption<Integer> KUBERNETES_MAX_RETRY_ATTEMPTS =
+		key("kubernetes.client.max-retry-attempts")
+			.intType()
+			.defaultValue(5)
+			.withDescription("Defines the number of Kubernetes resources update operation retries before the client " +
+				"gives up. For example, updating the ConfigMap.");
+

Review comment:
       I think "Kubernetes resources update operation" is a bit too general. E.g., one could argue that creating a new pod is also a "Kubernetes resources update operation".
   
   I would suggest `kubernetes.transactional-operation.max-retries` as the configuration key, and explain what is a transactional operation (a group of operations that are guaranteed atomic) in the description.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesConfigMapWatcher.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+
+import io.fabric8.kubernetes.api.model.ConfigMap;
+
+import java.util.Collections;
+
+/**
+ * Watcher for ConfigMaps in Kubernetes.
+ */
+public class KubernetesConfigMapWatcher extends KubernetesWatcher<ConfigMap, KubernetesConfigMap> {
+
+	public KubernetesConfigMapWatcher(FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> callbackHandler) {
+		super(callbackHandler);
+	}
+
+	@Override
+	public void eventReceived(Action action, ConfigMap configMap) {
+		logger.debug("Received {} event for configMap {}, details: {}",
+			action, configMap.getMetadata().getName(), configMap.getData());
+		switch (action) {
+			case ADDED:
+				callbackHandler.onAdded(Collections.singletonList(new KubernetesConfigMap(configMap)));
+				break;
+			case MODIFIED:
+				callbackHandler.onModified(Collections.singletonList(new KubernetesConfigMap(configMap)));
+				break;
+			case ERROR:
+				callbackHandler.onError(Collections.singletonList(new KubernetesConfigMap(configMap)));
+				break;
+			case DELETED:
+				callbackHandler.onDeleted(Collections.singletonList(new KubernetesConfigMap(configMap)));
+				break;

Review comment:
       Minor: we can deduplicate the codes by generating the singleton list before `switch`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesConfigMapWatcher.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+
+import io.fabric8.kubernetes.api.model.ConfigMap;
+
+import java.util.Collections;
+
+/**
+ * Watcher for ConfigMaps in Kubernetes.

Review comment:
       ```suggestion
    * Watcher for {@link ConfigMap}s in Kubernetes.
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesConfigMap.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.kubeclient.resources;
+
+import io.fabric8.kubernetes.api.model.ConfigMap;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Represent KubernetesConfigMap resource in kubernetes.

Review comment:
       ```suggestion
    * Represent {@link ConfigMap} resource in kubernetes.
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -104,6 +107,67 @@ KubernetesWatch watchPodsAndDoCallback(
 		Map<String, String> labels,
 		WatchCallbackHandler<KubernetesPod> podCallbackHandler);
 
+	/**
+	 * Create the ConfigMap with specified content. If the ConfigMap already exists, nothing will happen.
+	 *
+	 * @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 empty if the ConfigMap does not exist.

Review comment:
       ```suggestion
   	 * @return Return the ConfigMap, or empty if the ConfigMap does not exist.
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -104,6 +107,67 @@ KubernetesWatch watchPodsAndDoCallback(
 		Map<String, String> labels,
 		WatchCallbackHandler<KubernetesPod> podCallbackHandler);
 
+	/**
+	 * Create the ConfigMap with specified content. If the ConfigMap already exists, nothing will happen.
+	 *
+	 * @param configMap ConfigMap.
+	 *
+	 * @return Return the ConfigMap create future.
+	 */
+	CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap);

Review comment:
       I would suggest naming this method `createConfigMapIfAbsent`, to make it explicit that nothing will happen if the config map already exist.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,68 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		return CompletableFuture.runAsync(
+			() -> {
+				if (!getConfigMap(configMap.getName()).isPresent()) {
+					this.internalClient.configMaps().create(configMap.getInternalResource());
+				}
+			},
+			kubeClientExecutorService);

Review comment:
       Not related to this PR, but I think we can already replace `kubeClientExecutorService` with `AbstractResourceManagerDriver#ioExecutor`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -104,6 +107,67 @@ KubernetesWatch watchPodsAndDoCallback(
 		Map<String, String> labels,
 		WatchCallbackHandler<KubernetesPod> podCallbackHandler);
 
+	/**
+	 * Create the ConfigMap with specified content. If the ConfigMap already exists, nothing will happen.
+	 *
+	 * @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 empty if the ConfigMap does not exist.
+	 */
+	Optional<KubernetesConfigMap> getConfigMap(String name);
+
+	/**
+	 * Update an existing ConfigMap with the data.
+	 *
+	 * @param configMapName ConfigMap to be replaced with. 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_MAX_RETRY_ATTEMPTS}.
+	 * @param checker       Only the checker return true, the ConfigMap will be updated.
+	 * @param function      The obtained ConfigMap will be applied to this function and get a new one to replace.
+	 *
+	 * @return Return the ConfigMap update future.
+	 */
+	CompletableFuture<Boolean> checkAndUpdateConfigMap(
+		String configMapName,
+		Predicate<KubernetesConfigMap> checker,
+		FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function);

Review comment:
       I think we can get rid of the argument `checker`, and make `function` returns a `Optional<KubernetesConfigMap>` to indicate whether and how to update the config map. This should simplify the interface and make its contract easy to understand.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -104,6 +107,67 @@ KubernetesWatch watchPodsAndDoCallback(
 		Map<String, String> labels,
 		WatchCallbackHandler<KubernetesPod> podCallbackHandler);
 
+	/**
+	 * Create the ConfigMap with specified content. If the ConfigMap already exists, nothing will happen.
+	 *
+	 * @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 empty if the ConfigMap does not exist.
+	 */
+	Optional<KubernetesConfigMap> getConfigMap(String name);
+
+	/**
+	 * Update an existing ConfigMap with the data.
+	 *
+	 * @param configMapName ConfigMap to be replaced with. 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_MAX_RETRY_ATTEMPTS}.
+	 * @param checker       Only the checker return true, the ConfigMap will be updated.
+	 * @param function      The obtained ConfigMap will be applied to this function and get a new one to replace.
+	 *
+	 * @return Return the ConfigMap update future.
+	 */

Review comment:
       minor: It might be better to explain the contract details before the parameters, and keep the parameter docs brief.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,68 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		return CompletableFuture.runAsync(
+			() -> {
+				if (!getConfigMap(configMap.getName()).isPresent()) {
+					this.internalClient.configMaps().create(configMap.getInternalResource());
+				}
+			},
+			kubeClientExecutorService);
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			Predicate<KubernetesConfigMap> checker,
+			FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean shouldUpdate = checker.test(configMap);
+						if (!shouldUpdate) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						} else {
+							this.internalClient.configMaps()
+								.inNamespace(namespace)
+								.createOrReplace(function.apply(configMap).getInternalResource());
+						}
+						return shouldUpdate;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),
+				kubeClientExecutorService),
+			maxRetryAttempts,
+			kubeClientExecutorService);
+	}

Review comment:
       Combining `checker` and `function` into a function that returns `Optional<KubernetesConfigMap>` should help simplify this implementation.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java
##########
@@ -50,6 +52,8 @@
 	public static final String LABEL_COMPONENT_KEY = "component";
 	public static final String LABEL_COMPONENT_JOB_MANAGER = "jobmanager";
 	public static final String LABEL_COMPONENT_TASK_MANAGER = "taskmanager";
+	public static final String LABEL_CONFIGMAP_TYPE_KEY = "configmap-type";
+	public static final String LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY = "high-availability";

Review comment:
       This change does not belong to this commit.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java
##########
@@ -80,4 +84,14 @@
 	public static final String RESTART_POLICY_OF_NEVER = "Never";
 
 	public static final String NATIVE_KUBERNETES_COMMAND = "native-k8s";
+
+	// Constants for Kubernetes high availability
+	public static final String LEADER_ADDRESS_KEY = "address";
+	public static final String LEADER_SESSION_ID_KEY = "sessionId";
+	public static final String CHECKPOINT_COUNTER_KEY = "counter";
+	public static final String RUNNING_JOBS_REGISTRY_KEY_PREFIX = "runningJobsRegistry";
+	public static final String JOB_GRAPH_STORE_KEY_PREFIX = "jobGraph";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	public static final String LEADER_ANNOTATION_KEY = "control-plane.alpha.kubernetes.io/leader";

Review comment:
       These changes does not belong to this commit.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -104,6 +107,67 @@ KubernetesWatch watchPodsAndDoCallback(
 		Map<String, String> labels,
 		WatchCallbackHandler<KubernetesPod> podCallbackHandler);
 
+	/**
+	 * Create the ConfigMap with specified content. If the ConfigMap already exists, nothing will happen.
+	 *
+	 * @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 empty if the ConfigMap does not exist.
+	 */
+	Optional<KubernetesConfigMap> getConfigMap(String name);
+
+	/**
+	 * Update an existing ConfigMap with the data.
+	 *
+	 * @param configMapName ConfigMap to be replaced with. 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_MAX_RETRY_ATTEMPTS}.
+	 * @param checker       Only the checker return true, the ConfigMap will be updated.
+	 * @param function      The obtained ConfigMap will be applied to this function and get a new one to replace.
+	 *
+	 * @return Return the ConfigMap update future.
+	 */
+	CompletableFuture<Boolean> checkAndUpdateConfigMap(
+		String configMapName,
+		Predicate<KubernetesConfigMap> checker,
+		FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function);
+
+	/**
+	 * Watch the ConfigMaps with specified name and do the {@link WatchCallbackHandler}.
+	 *
+	 * @param name name to filter the ConfigMaps to watch
+	 * @param callbackHandler callbackHandler which reacts to ConfigMap events
+	 * @return Return a watch for ConfigMaps. It needs to be closed after use.
+	 */
+	KubernetesWatch watchConfigMapsAndDoCallback(

Review comment:
       ```suggestion
   	KubernetesWatch watchConfigMaps(
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	protected volatile LeaderContender leaderContender;
+
+	private volatile UUID issuedLeaderSessionID;
+
+	protected volatile UUID confirmedLeaderSessionID;
+
+	protected volatile String confirmedLeaderAddress;
+
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+		}
+
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		internalStop();
+	}

Review comment:
       I noticed that `internalStart` is called from inside the `synchronized` block, while `internalStop` is called from outside the `synchronized` block. I think this is a bit implicit and might become hard to maintain. Implementations extending this abstract class can easily overlook this difference.
   
   I wonder how does it hurt if we move `internalStop` to inside the `synchronized` block. It might hold the lock for longer while blocking other threads from accessing `synchronized` codes protected by this lock, which should be fine given that the service is stopped anyway. At meantime, we gain better maintainability from it.
   
   WDYT?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -196,6 +224,16 @@ public static String getCommonStartCommand(
 		).collect(Collectors.toList());
 	}
 
+	public static Predicate<KubernetesConfigMap> getLeaderChecker() {
+		return configMap -> {
+			if (configMap.getAnnotations() != null) {
+				final String leader = configMap.getAnnotations().get(LEADER_ANNOTATION_KEY);
+				return leader != null && leader.contains(LOCK_IDENTITY);
+			}
+			return false;
+		};
+	}

Review comment:
       This change is not a common config map operations. It is closely related to leader election. Thus, it does not belong to this commit.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -103,6 +121,52 @@ public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, WatchC
 		return watchPodsAndDoCallbackFunction.apply(labels, podCallbackHandler);
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		configMapStore.putIfAbsent(configMap.getName(), configMap);
+		return CompletableFuture.completedFuture(null);
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final KubernetesConfigMap configMap = configMapStore.get(name);
+		if (configMap == null) {
+			return Optional.empty();
+		}
+		return Optional.of(new MockKubernetesConfigMap(configMap.getName(), new HashMap<>(configMap.getData())));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			Predicate<KubernetesConfigMap> checker,
+			FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function) {
+		return getConfigMap(configMapName).map(FunctionUtils.uncheckedFunction(
+			configMap -> {
+				final boolean shouldUpdate = checker.test(configMap);
+				if (shouldUpdate) {
+					configMapStore.put(configMap.getName(), function.apply(configMap));
+				}
+				return CompletableFuture.completedFuture(shouldUpdate);
+			}))
+			.orElseThrow(() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists."));
+	}

Review comment:
       Not sure about having a `configMapStore` and these implementations in the testing class. I would suggest to have `createConfigMapFunction`, `getConfigMapFunction`, `checkAndUpdateConfigMapFunction` instead.
   
   The current limitation is less flexible. E.g., we cannot simulate situations where config maps are deleted externally.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	protected volatile LeaderContender leaderContender;
+
+	private volatile UUID issuedLeaderSessionID;
+
+	protected volatile UUID confirmedLeaderSessionID;
+
+	protected volatile String confirmedLeaderAddress;
+
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+		}
+
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		internalStop();
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (logger.isDebugEnabled()) {
+			logger.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		if (checkLeaderLatch()) {

Review comment:
       I think the concept 'leader latch' is from ZooKeeper? Maybe we should abstract this as a common meaningful interface?
   I noticed the only difference between `checkLeadeerLatch` and `hasLeadership` is `leaderSessionId.equals(issuedLeaderSessionID)`. If we adjust the order of the `if`s, would it be possible to get rid of `checkLeaderLatch` and replace it with an abstract `hasLeaderShip`?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.client.lease-duration")

Review comment:
       ```suggestion
   			key("high-availability.kubernetes.leader-election.lease-duration")
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.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.kubernetes.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.flink.kubernetes.utils.Constants.LOCK_IDENTITY;
+
+/**
+ * Represent Leader Elector in kubernetes.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {

Review comment:
       IIUC, the lifecycle for each `run` for the elector ends when the leadership is revoked. To join another round of election, we need to trigger `run` again. It would be better to explain these in the JavaDocs.  

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.leaderelection.TestingContender;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link KubernetesLeaderElectionService}.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+
+	private final ExecutorService executorService =
+		Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	private final Configuration configuration = new Configuration();
+
+	protected static final String CLUSTER_ID = "leader-test-cluster";
+	protected static final String LEADER_URL = "akka.tcp://flink@172.20.1.21:6123/user/rpc/resourcemanager";
+	protected static final long TIMEOUT = 30L * 1000L;
+	protected static final String LEADER_CONFIGMAP_NAME = "k8s-ha-app1-resourcemanager";
+	protected final Map<String, KubernetesConfigMap> configMapStore = new HashMap<>();
+	protected final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> configMapsAndDoCallbackFuture =
+		new CompletableFuture<>();
+	protected final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> leaderRetrievalConfigMapCallback =
+		new CompletableFuture<>();

Review comment:
       It seems `configMapStore` and the 2 futures are reused across test cases. There could be stability issues.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.client.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(30))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +
+				"continuously renew its lease time to indicate its existence. And the followers will do a lease " +
+				"checking against the current time. \"renewTime + leaseDuration > now\" means the leader is alive.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RENEW_DEADLINE =
+			key("high-availability.kubernetes.client.renew-deadline")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Defines the deadline when the leader tries to renew the lease in ms. If it could not " +
+				"succeed in the given time, the renew operation will be aborted.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RETRY_PERIOD =
+			key("high-availability.kubernetes.client.retry-period")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(3))

Review comment:
       Would it be too frequent to check the leadership every 3s? Given that the default lease duration is 30s.
   
   Ideally, if a contender checks the leadership and learns the remaining lease during, it does not make sense to check again before the lease can be expired. Maybe it makes sense to decide when to perform the next checking dynamically based on the remaining lease duration.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	protected volatile LeaderContender leaderContender;
+
+	private volatile UUID issuedLeaderSessionID;
+
+	protected volatile UUID confirmedLeaderSessionID;
+
+	protected volatile String confirmedLeaderAddress;
+
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+		}
+
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		internalStop();
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (logger.isDebugEnabled()) {
+			logger.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		if (checkLeaderLatch()) {
+			// check if this is an old confirmation call
+			synchronized (lock) {
+				if (running) {
+					if (leaderSessionID.equals(this.issuedLeaderSessionID)) {
+						confirmLeaderInformation(leaderSessionID, leaderAddress);
+						writeLeaderInformation();
+					}
+				} else {
+					logger.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"LeaderElectionService has already been stopped.", leaderSessionID);
+				}
+			}
+		} else {
+			logger.warn("The leader session ID {} was confirmed even though the " +
+				"corresponding JobManager was not elected as the leader.", leaderSessionID);
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		return checkLeaderLatch() && leaderSessionId.equals(issuedLeaderSessionID);
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	protected abstract void internalStart(LeaderContender contender) throws Exception;
+
+	protected abstract void internalStop() throws Exception;
+
+	protected abstract void writeLeaderInformation();
+
+	protected abstract boolean checkLeaderLatch();

Review comment:
       Would be nice to add JavaDocs for these interfaces. They are meant to be implemented by various leader election services.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesHighAvailabilityOptions;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.NAME_SEPARATOR;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * High availability service for Kubernetes.
+ */
+public class KubernetesHaServices implements HighAvailabilityServices {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesHaServices.class);
+
+	private static final String RESOURCE_MANAGER_NAME = "resourcemanager";
+
+	private static final String DISPATCHER_NAME = "dispatcher";
+
+	private static final String JOB_MANAGER_NAME = "jobmanager";
+
+	private static final String REST_SERVER_NAME = "restserver";
+
+	private final String leaderSuffix;
+
+	private final String clusterId;
+
+	/** Kubernetes client. */
+	private final FlinkKubeClient kubeClient;
+
+	/** The executor to run Kubernetes operations on. */
+	private final Executor executor;
+
+	/** The runtime configuration. */
+	private final Configuration configuration;
+
+	/** Store for arbitrary blobs. */
+	private final BlobStoreService blobStoreService;
+
+	/** The Kubernetes based running jobs registry. */
+	private final RunningJobsRegistry runningJobsRegistry;

Review comment:
       These are in-common with `ZooKeeperHaService`. Would it make sense to abstract them to an `AbstractHaService`?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,68 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		return CompletableFuture.runAsync(
+			() -> {
+				if (!getConfigMap(configMap.getName()).isPresent()) {
+					this.internalClient.configMaps().create(configMap.getInternalResource());
+				}

Review comment:
       The existence check and creation are not guaranteed atomic. What happens if another client creates the config map in between? Does the creation operation fail or overwrites the existing one? If it fails, where is the exception handled?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,58 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateAndDeleteConfigMap() {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(haLabels);
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+
+		final Supplier<Exception> configMapNotExistException = () -> new Exception("ConfigMap not exist");
+		FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function = c -> {
+			c.getData().put(LEADER_ADDRESS_KEY, LEADER_ADDRESS_NEW);
+			return c;
+		};
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+
+		// Checker not pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(LEADER_CONFIG_MAP_NAME, c -> false, function).get();
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(LEADER_CONFIG_MAP_NAME, c -> true, function).get();
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS_NEW));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+	}
+
+	private KubernetesConfigMap buildHAConfigMap() {
+		final Map<String, String> data = new HashMap<>();
+		data.put(LEADER_ADDRESS_KEY, LEADER_ADDRESS);
+		return new KubernetesConfigMap(new ConfigMapBuilder()
+			.withNewMetadata()
+			.withName(LEADER_CONFIG_MAP_NAME)
+			.withLabels(haLabels)
+			.endMetadata()
+			.withData(data).build());
+	}

Review comment:
       I would suggest to use arbitrary testing config map for testing the client. The config map interfaces should work correctly regardless of whether the config map is for HA or not.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.client.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(30))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +
+				"continuously renew its lease time to indicate its existence. And the followers will do a lease " +
+				"checking against the current time. \"renewTime + leaseDuration > now\" means the leader is alive.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RENEW_DEADLINE =
+			key("high-availability.kubernetes.client.renew-deadline")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))

Review comment:
       I think the default 15s timeout does not match the default 30s lease duration. That means after the leader gives up the leadership, there are another 15s before another contender can become leader. During this 15s, there's practically no leader.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServicesFactory.java
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.kubeclient.KubeClientFactory;
+import org.apache.flink.runtime.blob.BlobUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesFactory;
+
+import java.util.concurrent.Executor;
+
+/**
+ * Factory for creating Kubernetes high availability services.
+ */
+public class KubernetesHaServicesFactory implements HighAvailabilityServicesFactory {
+
+	@Override
+	public HighAvailabilityServices createHAServices(Configuration configuration, Executor executor) throws Exception {
+		return new KubernetesHaServices(
+			KubeClientFactory.fromConfiguration(configuration),

Review comment:
       IIUC, this means we create 2 `KubeClient`s in the JobManager process? Would it be a problem?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");

Review comment:
       Any reason to make this suffix configurable?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -106,13 +110,37 @@ public static String getDeploymentName(String clusterId) {
 	 * @return Task manager labels.
 	 */
 	public static Map<String, String> getTaskManagerLabels(String clusterId) {
-		final Map<String, String> labels = new HashMap<>();
-		labels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE);
-		labels.put(Constants.LABEL_APP_KEY, clusterId);
+		final Map<String, String> labels = new HashMap<>(getCommonLabels(clusterId));
 		labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_TASK_MANAGER);
 		return Collections.unmodifiableMap(labels);
 	}
 
+	/**
+	 * Get the common labels for Flink native clusters. All the Kubernetes resources will be set with these labels.
+	 *
+	 * @param clusterId cluster id
+	 * @return Return common labels map
+	 */
+	public static Map<String, String> getCommonLabels(String clusterId) {
+		Map<String, String> commonLabels = new HashMap<>();
+		commonLabels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE);
+		commonLabels.put(Constants.LABEL_APP_KEY, clusterId);
+
+		return Collections.unmodifiableMap(commonLabels);

Review comment:
       Why returning an unmodifiable map? The returned map is a new instance generated in this method. There should be no other reference to it except for what returned from this method.
   
   Returning a modifiable map would also save us from converting back to a modifiable map in `getTaskManagerLabels` and `getConfigMapLabels`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java
##########
@@ -80,4 +84,14 @@
 	public static final String RESTART_POLICY_OF_NEVER = "Never";
 
 	public static final String NATIVE_KUBERNETES_COMMAND = "native-k8s";
+
+	// Constants for Kubernetes high availability
+	public static final String LEADER_ADDRESS_KEY = "address";
+	public static final String LEADER_SESSION_ID_KEY = "sessionId";
+	public static final String CHECKPOINT_COUNTER_KEY = "counter";
+	public static final String RUNNING_JOBS_REGISTRY_KEY_PREFIX = "runningJobsRegistry";
+	public static final String JOB_GRAPH_STORE_KEY_PREFIX = "jobGraph";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();

Review comment:
       This is not a typical "constant". It is expected to be different for each process. We'd better move it to somewhere else (e.g., `KubernetesLeaderElectionService`) and init it as a non static field.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java
##########
@@ -115,11 +115,7 @@ public String getImage() {
 
 	@Override
 	public Map<String, String> getCommonLabels() {
-		Map<String, String> commonLabels = new HashMap<>();
-		commonLabels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE);
-		commonLabels.put(Constants.LABEL_APP_KEY, getClusterId());
-
-		return Collections.unmodifiableMap(commonLabels);
+		return KubernetesUtils.getCommonLabels(getClusterId());

Review comment:
       Seems this change does not belong to this commit?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();

Review comment:
       It is not introduced by this PR, but it might be better to comment which internal states are protected by this lock.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,58 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateAndDeleteConfigMap() {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(haLabels);
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(false));
+	}

Review comment:
       Would be better to split this into 2 cases, keeping one purpose for each case.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,58 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateAndDeleteConfigMap() {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(haLabels);
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+
+		final Supplier<Exception> configMapNotExistException = () -> new Exception("ConfigMap not exist");
+		FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function = c -> {
+			c.getData().put(LEADER_ADDRESS_KEY, LEADER_ADDRESS_NEW);
+			return c;
+		};
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+
+		// Checker not pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(LEADER_CONFIG_MAP_NAME, c -> false, function).get();
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(LEADER_CONFIG_MAP_NAME, c -> true, function).get();
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS_NEW));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+	}

Review comment:
       1. Let's separate this to 2 test cases.
   2. Instead of checking the existence with `orElseThrow`, I think asserting `Optional#isPresent` should provide better readability, for all the 3 occurences.
   ```
   final Optional<ConfigMap> configMapOpt = flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME);
   assertThat(configMapOpt.isPresent(), true);
   assertThat(configMapOpt.get().getData().get(LEADER_ADDRESS_KEY), is(xxx));
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.client.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(30))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +
+				"continuously renew its lease time to indicate its existence. And the followers will do a lease " +
+				"checking against the current time. \"renewTime + leaseDuration > now\" means the leader is alive.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RENEW_DEADLINE =
+			key("high-availability.kubernetes.client.renew-deadline")

Review comment:
       ```suggestion
   			key("high-availability.kubernetes.leader-election.renew-deadline")
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.client.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(30))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +
+				"continuously renew its lease time to indicate its existence. And the followers will do a lease " +
+				"checking against the current time. \"renewTime + leaseDuration > now\" means the leader is alive.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RENEW_DEADLINE =
+			key("high-availability.kubernetes.client.renew-deadline")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Defines the deadline when the leader tries to renew the lease in ms. If it could not " +
+				"succeed in the given time, the renew operation will be aborted.");

Review comment:
       It's a bit confusing what does "renew operation will be aborted" mean. I think we should explain that a leader will give up its leadership if it cannot successfully renew the lease within this time.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionService.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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 java.util.List;
+import java.util.Map;
+import java.util.Objects;
+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.watchConfigMapsAndDoCallback(configMapName, new ConfigMapCallbackHandlerImpl());
+	}
+
+	@Override
+	public void internalStop() {
+		if (kubernetesWatch != null) {
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	protected void writeLeaderInformation() {
+		updateConfigMap(configMapName);
+	}
+
+	@Override
+	protected boolean checkLeaderLatch() {
+		return kubeClient.getConfigMap(configMapName)
+			.map(configMap -> KubernetesUtils.getLeaderChecker().test(configMap))
+			.orElse(false);
+	}
+
+	@Override
+	public String toString() {
+		return "KubernetesLeaderElectionService{configMapName='" + configMapName + "'}";
+	}
+
+	private void updateConfigMap(String configMapName) {

Review comment:
       I think the name `updateConfigMap` is not very descriptive. What this method really does is to write its own information to the leader information config map.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionService.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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 java.util.List;
+import java.util.Map;
+import java.util.Objects;
+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.watchConfigMapsAndDoCallback(configMapName, new ConfigMapCallbackHandlerImpl());
+	}
+
+	@Override
+	public void internalStop() {
+		if (kubernetesWatch != null) {
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	protected void writeLeaderInformation() {
+		updateConfigMap(configMapName);
+	}
+
+	@Override
+	protected boolean checkLeaderLatch() {
+		return kubeClient.getConfigMap(configMapName)
+			.map(configMap -> KubernetesUtils.getLeaderChecker().test(configMap))
+			.orElse(false);
+	}
+
+	@Override
+	public String toString() {
+		return "KubernetesLeaderElectionService{configMapName='" + configMapName + "'}";
+	}
+
+	private void updateConfigMap(String configMapName) {
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				KubernetesUtils.getLeaderChecker(),
+				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 configMap;
+				}).get();
+		} catch (Exception e) {
+			leaderContender.handleError(new Exception("Could not update ConfigMap " + configMapName, e));
+		}
+	}
+
+	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,
+					KubernetesUtils.getLeaderChecker(),
+					configMap -> {
+						configMap.getData().remove(LEADER_ADDRESS_KEY);
+						configMap.getData().remove(LEADER_SESSION_ID_KEY);
+						return 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 (checkLeaderLatch()) {
+				configMaps.forEach(configMap -> {
+					if (isLeaderChanged(configMap)) {
+						// the data field does not correspond to the expected leader information
+						if (logger.isDebugEnabled()) {
+							logger.debug("Correcting leader information in {} by {}.",
+								configMapName, leaderContender.getDescription());
+						}
+						updateConfigMap(configMap.getName());
+					}
+				});
+			}
+		}

Review comment:
       Would it be possible that this method is called before `confirmLeadership`, so that the leader election service writes the leader information before the contender confirms the leadership?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalService.java
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+
+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;
+
+/**
+ * The counterpart to the {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionService}.
+ * This implementation of the {@link LeaderRetrievalService} retrieves the current leader which has
+ * been elected by the {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionService}.
+ * The leader address as well as the current leader session ID is retrieved from Kubernetes ConfigMap.
+ */
+class KubernetesLeaderRetrievalService implements LeaderRetrievalService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderRetrievalService.class);
+
+	private final Object lock = new Object();

Review comment:
       Better to explain which states/variables should be guarded by this lock.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesHighAvailabilityOptions;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.NAME_SEPARATOR;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * High availability service for Kubernetes.
+ */
+public class KubernetesHaServices implements HighAvailabilityServices {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesHaServices.class);
+
+	private static final String RESOURCE_MANAGER_NAME = "resourcemanager";
+
+	private static final String DISPATCHER_NAME = "dispatcher";
+
+	private static final String JOB_MANAGER_NAME = "jobmanager";
+
+	private static final String REST_SERVER_NAME = "restserver";
+
+	private final String leaderSuffix;
+
+	private final String clusterId;
+
+	/** Kubernetes client. */
+	private final FlinkKubeClient kubeClient;
+
+	/** The executor to run Kubernetes operations on. */
+	private final Executor executor;
+
+	/** The runtime configuration. */
+	private final Configuration configuration;
+
+	/** Store for arbitrary blobs. */
+	private final BlobStoreService blobStoreService;
+
+	/** The Kubernetes based running jobs registry. */
+	private final RunningJobsRegistry runningJobsRegistry;
+
+	KubernetesHaServices(
+			FlinkKubeClient kubeClient,
+			Executor executor,
+			Configuration config,
+			BlobStoreService blobStoreService) {
+
+		this.kubeClient = checkNotNull(kubeClient);
+		this.executor = checkNotNull(executor);
+		this.configuration = checkNotNull(config);
+		this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID));
+		this.blobStoreService = blobStoreService;
+
+		this.leaderSuffix = config.getString(KubernetesHighAvailabilityOptions.HA_KUBERNETES_LEADER_SUFFIX);
+
+		this.runningJobsRegistry = new StandaloneRunningJobsRegistry();
+	}
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		return createLeaderRetrievalService(RESOURCE_MANAGER_NAME);
+	}
+
+	@Override
+	public LeaderRetrievalService getDispatcherLeaderRetriever() {
+		return createLeaderRetrievalService(DISPATCHER_NAME);
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		return createLeaderRetrievalService(getLeaderNameForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress) {
+		return getJobManagerLeaderRetriever(jobID);
+	}
+
+	@Override
+	public LeaderRetrievalService getClusterRestEndpointLeaderRetriever() {
+		return createLeaderRetrievalService(REST_SERVER_NAME);
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		return createLeaderElectionService(RESOURCE_MANAGER_NAME);
+	}
+
+	@Override
+	public LeaderElectionService getDispatcherLeaderElectionService() {
+		return createLeaderElectionService(DISPATCHER_NAME);
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		return createLeaderElectionService(getLeaderNameForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderElectionService getClusterRestEndpointLeaderElectionService() {
+		return createLeaderElectionService(REST_SERVER_NAME);
+	}
+
+	@Override
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
+		return new StandaloneCheckpointRecoveryFactory();
+	}
+
+	@Override
+	public JobGraphStore getJobGraphStore() {
+		return new StandaloneJobGraphStore();
+	}
+
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() {
+		return runningJobsRegistry;
+	}
+
+	@Override
+	public BlobStore createBlobStore() {
+		return blobStoreService;
+	}
+
+	@Override
+	public void close() throws Exception {
+		Throwable exception = null;
+
+		try {
+			blobStoreService.close();
+		} catch (Throwable t) {
+			exception = t;
+		}
+
+		kubeClient.close();
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(exception, "Could not properly close the KubernetesHaServices.");
+		}
+	}
+
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		LOG.info("Close and clean up all data for KubernetesHaServices.");
+
+		Throwable exception = null;
+
+		try {
+			blobStoreService.closeAndCleanupAllData();
+		} catch (Throwable t) {
+			exception = t;
+		}
+
+		try {
+			kubeClient.deleteConfigMapsByLabels(
+				KubernetesUtils.getConfigMapLabels(clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+		} catch (Throwable t) {
+			exception = ExceptionUtils.firstOrSuppressed(t, exception);
+		}
+
+		kubeClient.close();
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(
+				exception, "Could not properly close and clean up all data of KubernetesHaServices.");
+		}
+		LOG.info("Finished cleaning up the high availability data.");
+	}
+
+	private KubernetesLeaderElectionService createLeaderElectionService(String leaderName) {
+		return new KubernetesLeaderElectionService(
+			kubeClient,
+			executor,
+			KubernetesLeaderElectionConfiguration.fromConfiguration(getLeaderConfigMapName(leaderName), configuration));
+	}
+
+	private KubernetesLeaderRetrievalService createLeaderRetrievalService(String leaderName) {
+		return new KubernetesLeaderRetrievalService(kubeClient, getLeaderConfigMapName(leaderName));
+	}

Review comment:
       I think we can make these two methods common interfaces in `AbstractHaService`, and provide different implementations for `Kubernetes/ZooKeeperHaService`. We can also move the component names to the common base class, and converting them to lock path in `ZooKeeperHaService#createLeaderElection/RetrievalService`. In this way, all the `getXXXLeaderElectionService/Retriever` methods can be moved to the base class and reused.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesWatcher.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.client.KubernetesClientException;
+import io.fabric8.kubernetes.client.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Watcher for resources in Kubernetes.
+ */
+public abstract class KubernetesWatcher<T extends HasMetadata, K extends KubernetesResource<T>> implements Watcher<T> {

Review comment:
       It would be better to place changes in this class and `KubernetesPodsWatcher` in a separate commit. It seems to me these changes are re-abstraction of the watchers, not watcher callbacks as described in the commit message.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.client.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(30))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +
+				"continuously renew its lease time to indicate its existence. And the followers will do a lease " +
+				"checking against the current time. \"renewTime + leaseDuration > now\" means the leader is alive.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RENEW_DEADLINE =
+			key("high-availability.kubernetes.client.renew-deadline")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Defines the deadline when the leader tries to renew the lease in ms. If it could not " +
+				"succeed in the given time, the renew operation will be aborted.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RETRY_PERIOD =
+			key("high-availability.kubernetes.client.retry-period")

Review comment:
       ```suggestion
   			key("high-availability.kubernetes.leader-election.retry-period")
   ```

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.leaderelection.TestingContender;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link KubernetesLeaderElectionService}.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+
+	private final ExecutorService executorService =
+		Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	private final Configuration configuration = new Configuration();
+
+	protected static final String CLUSTER_ID = "leader-test-cluster";
+	protected static final String LEADER_URL = "akka.tcp://flink@172.20.1.21:6123/user/rpc/resourcemanager";
+	protected static final long TIMEOUT = 30L * 1000L;
+	protected static final String LEADER_CONFIGMAP_NAME = "k8s-ha-app1-resourcemanager";
+	protected final Map<String, KubernetesConfigMap> configMapStore = new HashMap<>();
+	protected final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> configMapsAndDoCallbackFuture =
+		new CompletableFuture<>();
+	protected final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> leaderRetrievalConfigMapCallback =
+		new CompletableFuture<>();
+
+	@Before
+	public void setup() {
+		configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
+	}
+
+	@After
+	public void teardown() {
+		executorService.shutdownNow();
+	}
+
+	protected KubernetesLeaderElectionService createLeaderElectionService(AtomicBoolean leaderController) {
+		final TestingFlinkKubeClient flinkKubeClient = TestingFlinkKubeClient.builder()
+			.setConfigMapStore(configMapStore)
+			.setWatchConfigMapsAndDoCallbackFunction((ignore, handler) -> {
+				configMapsAndDoCallbackFuture.complete(handler);
+				return new TestingFlinkKubeClient.MockKubernetesWatch();
+			})
+			.setLeaderController(leaderController).build();
+		return new KubernetesLeaderElectionService(
+			flinkKubeClient,
+			executorService,
+			KubernetesLeaderElectionConfiguration.fromConfiguration(LEADER_CONFIGMAP_NAME, configuration));
+	}
+
+	protected KubernetesLeaderRetrievalService createLeaderRetrievalService() {
+		final TestingFlinkKubeClient flinkKubeClient = TestingFlinkKubeClient.builder()
+			.setConfigMapStore(configMapStore)
+			.setWatchConfigMapsAndDoCallbackFunction((ignore, handler) -> {
+				leaderRetrievalConfigMapCallback.complete(handler);
+				return new TestingFlinkKubeClient.MockKubernetesWatch();
+			}).build();
+		return new KubernetesLeaderRetrievalService(flinkKubeClient, LEADER_CONFIGMAP_NAME);
+	}
+
+	/**
+	 * Context to leader election and retrieval tests.
+	 */
+	protected class Context {
+		final AtomicBoolean leaderController = new AtomicBoolean(false);
+		final KubernetesLeaderElectionService leaderElectionService = createLeaderElectionService(leaderController);
+		final TestingContender contender = new TestingContender(LEADER_URL, leaderElectionService);
+
+		final KubernetesLeaderRetrievalService leaderRetrievalService = createLeaderRetrievalService();
+		final TestingListener listener = new TestingListener();
+
+		protected final void runTest(RunnableWithException testMethod) throws Exception {
+			leaderElectionService.start(contender);
+			leaderController.set(true);
+			contender.waitForLeader(TIMEOUT);
+			assertThat(contender.isLeader(), is(true));
+			leaderRetrievalService.start(listener);
+			testMethod.run();
+			leaderElectionService.stop();
+			leaderRetrievalService.stop();
+		}

Review comment:
       I would suggest to separate the action granting leadership to the contender apart from this method, for better readability.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader

Review comment:
       I think regex before the examples would help understand the pattern.
   IIUC, `k8s-ha-<cluster-id>-<component-name>-leader`?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.leaderelection.TestingContender;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link KubernetesLeaderElectionService}.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+
+	private final ExecutorService executorService =
+		Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	private final Configuration configuration = new Configuration();

Review comment:
       It seems `executorService ` and `configuration` are reused across test cases. There could be stability issues.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.leaderelection.TestingContender;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link KubernetesLeaderElectionService}.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+
+	private final ExecutorService executorService =
+		Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	private final Configuration configuration = new Configuration();
+
+	protected static final String CLUSTER_ID = "leader-test-cluster";
+	protected static final String LEADER_URL = "akka.tcp://flink@172.20.1.21:6123/user/rpc/resourcemanager";
+	protected static final long TIMEOUT = 30L * 1000L;
+	protected static final String LEADER_CONFIGMAP_NAME = "k8s-ha-app1-resourcemanager";
+	protected final Map<String, KubernetesConfigMap> configMapStore = new HashMap<>();
+	protected final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> configMapsAndDoCallbackFuture =
+		new CompletableFuture<>();
+	protected final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> leaderRetrievalConfigMapCallback =
+		new CompletableFuture<>();
+
+	@Before
+	public void setup() {
+		configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
+	}
+
+	@After
+	public void teardown() {
+		executorService.shutdownNow();
+	}
+
+	protected KubernetesLeaderElectionService createLeaderElectionService(AtomicBoolean leaderController) {
+		final TestingFlinkKubeClient flinkKubeClient = TestingFlinkKubeClient.builder()
+			.setConfigMapStore(configMapStore)
+			.setWatchConfigMapsAndDoCallbackFunction((ignore, handler) -> {
+				configMapsAndDoCallbackFuture.complete(handler);
+				return new TestingFlinkKubeClient.MockKubernetesWatch();
+			})
+			.setLeaderController(leaderController).build();
+		return new KubernetesLeaderElectionService(
+			flinkKubeClient,
+			executorService,
+			KubernetesLeaderElectionConfiguration.fromConfiguration(LEADER_CONFIGMAP_NAME, configuration));
+	}
+
+	protected KubernetesLeaderRetrievalService createLeaderRetrievalService() {
+		final TestingFlinkKubeClient flinkKubeClient = TestingFlinkKubeClient.builder()
+			.setConfigMapStore(configMapStore)
+			.setWatchConfigMapsAndDoCallbackFunction((ignore, handler) -> {
+				leaderRetrievalConfigMapCallback.complete(handler);
+				return new TestingFlinkKubeClient.MockKubernetesWatch();
+			}).build();
+		return new KubernetesLeaderRetrievalService(flinkKubeClient, LEADER_CONFIGMAP_NAME);
+	}
+
+	/**
+	 * Context to leader election and retrieval tests.
+	 */
+	protected class Context {
+		final AtomicBoolean leaderController = new AtomicBoolean(false);
+		final KubernetesLeaderElectionService leaderElectionService = createLeaderElectionService(leaderController);
+		final TestingContender contender = new TestingContender(LEADER_URL, leaderElectionService);
+
+		final KubernetesLeaderRetrievalService leaderRetrievalService = createLeaderRetrievalService();
+		final TestingListener listener = new TestingListener();
+
+		protected final void runTest(RunnableWithException testMethod) throws Exception {
+			leaderElectionService.start(contender);
+			leaderController.set(true);
+			contender.waitForLeader(TIMEOUT);
+			assertThat(contender.isLeader(), is(true));
+			leaderRetrievalService.start(listener);
+			testMethod.run();
+			leaderElectionService.stop();
+			leaderRetrievalService.stop();
+		}

Review comment:
       Or maybe renaming the method to `runTestAndGrantLeadershipToContender`, if this is needed for almost every test case.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.client.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(30))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +
+				"continuously renew its lease time to indicate its existence. And the followers will do a lease " +
+				"checking against the current time. \"renewTime + leaseDuration > now\" means the leader is alive.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RENEW_DEADLINE =
+			key("high-availability.kubernetes.client.renew-deadline")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Defines the deadline when the leader tries to renew the lease in ms. If it could not " +
+				"succeed in the given time, the renew operation will be aborted.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RETRY_PERIOD =
+			key("high-availability.kubernetes.client.retry-period")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(3))
+			.withDescription("Defines the pause between consecutive retries in ms. Both the leader and followers use " +
+				"this value for the retry.");

Review comment:
       It's not clear what a retry means. I think we should explain that all contenders periodically try to acquire/renew the leadership if possible, at this interval.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();

Review comment:
       IIUC, read or write all the `volatile` fields should be performed in a `synchronized` block protected by this lock? Then why `internalStart` also guarded by this lock?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,58 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateAndDeleteConfigMap() {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(haLabels);
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(false));
+	}

Review comment:
       I think we should also verify that `createConfigMap` does not overwrite an existing config map.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513513469



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}", this);
+
+			client.getUnhandledErrorListenable().removeListener(this);
+
+			client.getConnectionStateListenable().removeListener(listener);
+
+			Exception exception = null;
+
+			try {
+				cache.close();
+			} catch (Exception e) {
+				exception = e;
+			}
+
+			try {
+				leaderLatch.close();
+			} catch (Exception e) {
+				exception = ExceptionUtils.firstOrSuppressed(e, exception);
+			}
+
+			if (exception != null) {
+				throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		synchronized (lock) {
+			if (running) {
+				leaderElectionEventHandler.onGrantLeadership();
+			}
+		}

Review comment:
       I think I have given you bad advice here. Executing the callbacks under the inner `lock` is problematic if there is an outer call which holds the outer lock and calls into this class trying to obtain the inner `lock` (e.g. when concurrently closing this instance). Maybe we have to say in the contract of `ZooKeeperLeaderElectionDriver `, that spurious callbacks to `onGrantLeadership` can happen even after the driver is stopped and it is the responsibility of the implementor to filter invalid calls out (e.g. after close has been called on the outer class).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509044171



##########
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:
       What is throwing the exception?  If we only modify the given `ConfigMap`, then I don't see that we have to throw an exception. In any case, this should be added to the JavaDocs.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e33c52394024e8f1312c096a4b612d53d8848b97 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * f6be1c1045c89a0c2a2d8f1d317d41a656cae538 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895) 
   * 5e2cfa21731b9fb5dd976cce821bfda78daa91b7 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509055474



##########
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:
       Isn't the `KubernetesLeaderElector` responsible for creating the initial `configMap`? If this is the case, then I would assume that this component should also be responsible for ensuring that this map gets recreated in case of a deletion.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r506243026



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,68 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		return CompletableFuture.runAsync(
+			() -> {
+				if (!getConfigMap(configMap.getName()).isPresent()) {
+					this.internalClient.configMaps().create(configMap.getInternalResource());
+				}

Review comment:
       If the ConfigMap already exists, the create operation will fail. Since the existing ConfigMap may be created externally. So I will throw a `FlinkRuntimeException` when creating ConfigMap failed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514292977



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// Get the updated ConfigMap with new leader information
+						if (confirmedLeaderAddress != null && confirmedLeaderSessionID != null) {

Review comment:
       No, `leaderInformation` could be empty if we want to clear the old leader information.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509238124



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	/** The executor to run external IO operations on. */
+	protected final Executor executor;
+
+	/** The runtime configuration. */
+	protected final Configuration configuration;
+
+	/** Store for arbitrary blobs. */
+	private final BlobStoreService blobStoreService;
+
+	public AbstractHaServices(Executor executor, Configuration config, BlobStoreService blobStoreService) {
+
+		this.executor = checkNotNull(executor);
+		this.configuration = checkNotNull(config);
+		this.blobStoreService = blobStoreService;
+	}
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForResourceManager());
+	}
+
+	@Override
+	public LeaderRetrievalService getDispatcherLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForDispatcher());
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		return createLeaderRetrievalService(getLeaderNameForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress) {
+		return getJobManagerLeaderRetriever(jobID);
+	}
+
+	@Override
+	public LeaderRetrievalService getClusterRestEndpointLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForRestServer());
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForResourceManager());
+	}
+
+	@Override
+	public LeaderElectionService getDispatcherLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForDispatcher());
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		return createLeaderElectionService(getLeaderNameForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderElectionService getClusterRestEndpointLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForRestServer());
+	}
+
+	@Override
+	public BlobStore createBlobStore() {
+		return blobStoreService;
+	}
+
+	@Override
+	public void close() throws Exception {
+		Throwable exception = null;
+
+		try {
+			blobStoreService.close();
+		} catch (Throwable t) {
+			exception = t;
+		}
+
+		internalClose();
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(exception, "Could not properly close the " + getClass().getSimpleName());
+		}
+	}
+
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		logger.info("Close and clean up all data for {}.", getClass().getSimpleName());
+
+		Throwable exception = null;
+
+		try {
+			blobStoreService.closeAndCleanupAllData();
+		} catch (Throwable t) {
+			exception = t;
+		}
+
+		try {
+			internalCleanup();
+		} catch (Throwable t) {
+			exception = ExceptionUtils.firstOrSuppressed(t, exception);
+		}
+
+		internalClose();
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(
+				exception, "Could not properly close and clean up all data of high availability service.");
+		}
+		logger.info("Finished cleaning up the high availability data.");
+	}
+
+	/**
+	 * Create leader election service with specified leaderName.
+	 * @param leaderName ConfigMap name in Kubernetes or child node path in Zookeeper.
+	 *
+	 * @return Return LeaderElectionService using Zookeeper or Kubernetes.
+	 */
+	protected abstract LeaderElectionService createLeaderElectionService(String leaderName);
+
+	/**
+	 * Create leader retrieval service with specified leaderName.
+	 * @param leaderName ConfigMap name in Kubernetes or child node path in Zookeeper.
+	 *
+	 * @return Return LeaderRetrievalService using Zookeeper or Kubernetes.
+	 */
+	protected abstract LeaderRetrievalService createLeaderRetrievalService(String leaderName);
+
+	/**
+	 * Closes components which don't distinguish between close and closeAndCleanupAllData.
+	 */
+	protected abstract void internalClose() throws Exception;

Review comment:
       I will remove the `Exception` here since we do not throw in `FlinkKubeClient#close` and `CuratorFramework#close`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508986296



##########
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:
       Since the leader ConfigMap may be updated or deleted externally by the user, I introduce the `Watcher` in `KubernetesLeaderElectionService` for this situation. When it is deleted or wrongly updated, we will create a new one or correct the content.
   
   I think you mean that we do not need to handle this situation and just throw a fatal exception and failed the Flink cluster. Right?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 7f34f6f31596832223b1cf2010bed1c47f7502ea Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507653504



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.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.kubernetes.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.flink.kubernetes.utils.Constants.LOCK_IDENTITY;
+
+/**
+ * Represent Leader Elector in kubernetes.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {

Review comment:
       Yes. The `run` is a blocking call and will finished after `notLeader` callback is called. I will add some description in the java doc.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514073181



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/itcases/KubernetesLeaderElectionITCase.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.itcases;
+
+import org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionDriver;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.TestingContender;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+
+import org.junit.Test;
+
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_CONFIGMAP_NAME;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_URL;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.createLeaderElectionService;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.createLeaderRetrievalService;
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for the {@link DefaultLeaderElectionService} with {@link KubernetesLeaderElectionDriver} and
+ * {@link DefaultLeaderRetrievalService} with {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderRetrievalDriver}.
+ */
+public class KubernetesLeaderElectionITCase extends KubernetesITTestBase {

Review comment:
       I will add a ITCase for multiple `KubernetesLeaderElector` contending the leader to replace this one. Also add a simple leader election ITCase where we expect that a contender will become the leader and that we can retrieve its address from K8s.
   
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516800376



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderInformation.java
##########
@@ -74,7 +78,7 @@ public int hashCode() {
 		return result;
 	}
 
-	public static LeaderInformation known(@Nullable UUID leaderSessionID, @Nullable String leaderAddress) {
+	public static LeaderInformation known(UUID leaderSessionID, String leaderAddress) {

Review comment:
       add assert that `leaderSessionID` and `leaderAddress` is not null. Otherwise this method will allow us to break `isEmpty`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java
##########
@@ -189,7 +190,7 @@ public void testStopAtNonRetryableException() {
 			retryFuture.get();
 			fail("Exception should be thrown.");
 		} catch (Exception ex) {
-			assertThat(ExceptionUtils.findThrowableWithMessage(ex, notRetryExceptionMsg).isPresent(), is(true));
+			assertThat(ex, FlinkMatchers.containsCause(new FlinkRuntimeException(notRetryExceptionMsg)));

Review comment:
       nit: `new FlinkRuntimeException(notRetryExceptionMsg)` could be deduplicated.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -177,25 +178,32 @@ public void testOldConfirmLeaderInformation() throws Exception {
 	public void testErrorForwarding() throws Exception {
 		new Context() {{
 			runTest(() -> {
-				final Exception testException = new Exception("test leader Exeption");
+				final Exception testException = new Exception("test leader exception");
 
-				testingLeaderElectionDriver.getFatalErrorHandler().onFatalError(testException);
+				testingLeaderElectionDriver.onFatalError(testException);
 
 				testingContender.waitForError(timeout);
 				assertThat(testingContender.getError(), is(notNullValue()));
-				assertThat(testingContender.getError().getMessage(), containsString(testException.getMessage()));
+				assertThat(testingContender.getError(), FlinkMatchers.containsCause(testException));
 			});
 		}};
 	}
 
 	@Test
-	public void testErrorHappenAfterStop() throws Exception {
+	public void testErrorIsIgnoredAfterBeingStop() throws Exception {
 		new Context() {{
 			runTest(() -> {
-				final Exception testException = new Exception("test leader Exeption");
+				final Exception testException = new Exception("test leader exception");
 
 				leaderElectionService.stop();
-				testingLeaderElectionDriver.getFatalErrorHandler().onFatalError(testException);
+				testingLeaderElectionDriver.onFatalError(testException);
+
+				try {
+					testingContender.waitForError(timeout);

Review comment:
       Is the timeout really 3 seconds long? This might be a bit long for this test case. I would suggest to wait at most 100 or even 50 ms.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientITCase.java
##########
@@ -107,7 +108,7 @@ public void testCheckAndUpdateConfigMapConcurrently() throws Exception {
 						// noop
 					}
 				}
-			}, kubernetesResource.getExecutorService()));
+			}));

Review comment:
       I think we should not use the `ForkJoinPool#commonPool` here. Instead I would suggest to start a testing executor.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java
##########
@@ -75,24 +77,31 @@ public void testNotifyLeaderAddressEmpty() throws Exception {
 	public void testErrorForwarding() throws Exception {
 		new Context() {{
 			runTest(() -> {
-				final Exception testException = new Exception("test Exeption");
+				final Exception testException = new Exception("test exception");
 
-				testingLeaderRetrievalDriver.getFatalErrorHandler().onFatalError(testException);
+				testingLeaderRetrievalDriver.onFatalError(testException);
 
-				assertThat(testingListener.getError().getMessage(), containsString(testException.getMessage()));
+				testingListener.waitForError(timeout);
+				assertThat(testingListener.getError(), FlinkMatchers.containsCause(testException));
 			});
 		}};
 	}
 
 	@Test
-	public void testErrorHappenAfterStop() throws Exception {
+	public void testErrorIsIgnoredAfterBeingStop() throws Exception {
 		new Context() {{
 			runTest(() -> {
-				final Exception testException = new Exception("test Exeption");
+				final Exception testException = new Exception("test exception");
 
 				leaderRetrievalService.stop();
-				testingLeaderRetrievalDriver.getFatalErrorHandler().onFatalError(testException);
-
+				testingLeaderRetrievalDriver.onFatalError(testException);
+
+				try {
+					testingListener.waitForError(timeout);

Review comment:
       Same here with the timeout.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java
##########
@@ -118,7 +122,9 @@ public void testWriteLeaderInformationWhenConfigMapNotExist() throws Exception {
 					final String errorMsg = "Could not write leader information since ConfigMap "
 						+ LEADER_CONFIGMAP_NAME + " does not exist.";
 					assertThat(electionEventHandler.getError(), is(notNullValue()));
-					assertThat(electionEventHandler.getError().getMessage(), containsString(errorMsg));
+					assertThat(
+						electionEventHandler.getError(),
+						FlinkMatchers.containsCause(new KubernetesException(errorMsg)));

Review comment:
       Maybe it makes sense to add a new matcher which only checks the message: `FlinkMatchers.containsMessage(errorMsg)`. That way the test would not depend on the actually thrown exception type (here `KubernetesException`).

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java
##########
@@ -80,21 +75,25 @@ public void testMultipleKubernetesLeaderElectors() throws Exception {
 			}
 
 			// Wait for the first leader
-			final String firstLockIdentity = leaderStore.poll(TIMEOUT, TimeUnit.MILLISECONDS);
+			final String firstLockIdentity = TestingLeaderCallbackHandler.waitUntilNewLeaderAppear(TIMEOUT);

Review comment:
       nit: `waitUntilNewLeaderAppears`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514177126



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/itcases/KubernetesLeaderElectionITCase.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.itcases;
+
+import org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionDriver;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.TestingContender;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+
+import org.junit.Test;
+
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_CONFIGMAP_NAME;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_URL;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.createLeaderElectionService;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.createLeaderRetrievalService;
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for the {@link DefaultLeaderElectionService} with {@link KubernetesLeaderElectionDriver} and
+ * {@link DefaultLeaderRetrievalService} with {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderRetrievalDriver}.
+ */
+public class KubernetesLeaderElectionITCase extends KubernetesITTestBase {
+
+	/**
+	 * Start multiple leaders concurrently, if current leader dies, a new one could take over and update the leader
+	 * information successfully. The leader retrieval service should notice this change and notify listener with new
+	 * leader address.
+	 */
+	@Test
+	public void testConcurrentLeaderElectionAndRetrieval() throws Exception {

Review comment:
       Directly test the multiple `KubernetesLeaderElector` is more targeted. I will make this test as a simple leader election ITCase where we expect that a contender will become the leader and that we can retrieve its address from K8s.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514292399



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);

Review comment:
       I will add test for what I have said above. `DefaultLeaderElectionService` should ignore the late event(happen after closed).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 5e2cfa21731b9fb5dd976cce821bfda78daa91b7 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186) 
   * 7ae53916e13b43e94621217d7a354a672077e0ab Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285) 
   * aad1ba94672d89de5a4fc814297f0c40e3552db0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 37efc6f14c55e2bd261b70924c9525c6539447f8 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 5e2cfa21731b9fb5dd976cce821bfda78daa91b7 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186) 
   * 7ae53916e13b43e94621217d7a354a672077e0ab Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8720",
       "triggerID" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753",
       "triggerID" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884",
       "triggerID" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * ed02e962a354185fb37eaedbdfee217d691bd16b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509059229



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						}
+						return updated;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),

Review comment:
       I think this method is also lacking test coverage for exactly a case where `getConfigMap` returns `Optional.empty` or where
   
   ```
   this.internalClient.configMaps()
   	.inNamespace(namespace)
           .createOrReplace(updatedConfigMap.getInternalResource());
   ```
   
   fails.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514297081



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Base class for high availability unit tests with a configured testing Kubernetes client.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+	private static final String CLUSTER_ID = "leader-test-cluster";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	public static final String LEADER_URL = "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher";
+	public static final String LEADER_CONFIGMAP_NAME = "leader-test-cluster";
+
+	protected static final long TIMEOUT = 30L * 1000L;
+
+	protected ExecutorService executorService;
+	protected Configuration configuration;
+
+	@Before
+	public void setup() {
+		configuration = new Configuration();
+		configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
+		executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	}
+
+	@After
+	public void teardown() throws Exception {
+		executorService.shutdownNow();
+		executorService.awaitTermination(TIMEOUT, TimeUnit.MILLISECONDS);
+	}
+
+	/**
+	 * Context to leader election and retrieval tests.
+	 */
+	protected class Context {
+		/**
+		 * The configMapStore should only be used for set the following functions. Use a ConcurrentHashMap here since it
+		 * could be updated by {@link TestingFlinkKubeClient.MockKubernetesLeaderElector} and
+		 * {@link KubernetesLeaderElectionDriver}.
+		 */
+		private final Map<String, KubernetesConfigMap> configMapStore = new ConcurrentHashMap<>();
+
+		final AtomicBoolean leaderController = new AtomicBoolean(false);
+
+		final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+			configMapCallbackFutures = new ArrayList<>();
+
+		final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture = new CompletableFuture<>();
+		final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+		final FlinkKubeClient flinkKubeClient;
+
+		final LeaderElectionDriver leaderElectionDriver;
+		final TestingLeaderElectionEventHandler electionEventHandler;
+
+		final LeaderRetrievalDriver leaderRetrievalDriver;
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler;
+
+		Context() {
+			flinkKubeClient = getFlinkKubeClient();
+
+			electionEventHandler = new TestingLeaderElectionEventHandler(LEADER_URL);
+			leaderElectionDriver = createLeaderElectionDriver();
+
+			retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+			leaderRetrievalDriver = createLeaderRetrievalDriver();
+		}
+
+		void runTestAndGrantLeadershipToContender(RunnableWithException testMethod) throws Exception {
+			// Grant leadership
+			leaderController.set(true);
+			electionEventHandler.waitForLeader(TIMEOUT);
+			assertThat(electionEventHandler.isLeader(), is(true));
+
+			testMethod.run();
+
+			leaderElectionDriver.close();
+			leaderRetrievalDriver.close();
+		}
+
+		KubernetesConfigMap getLeaderConfigMap() {
+			final Optional<KubernetesConfigMap> configMapOpt = flinkKubeClient.getConfigMap(LEADER_CONFIGMAP_NAME);
+			assertThat(configMapOpt.isPresent(), is(true));
+			return configMapOpt.get();
+		}
+
+		protected FlinkKubeClient getFlinkKubeClient() {
+			return TestingFlinkKubeClient.builder()
+				.setCreateConfigMapFunction(configMap -> {
+					configMapStore.put(configMap.getName(), configMap);
+					return CompletableFuture.completedFuture(null);
+				})
+				.setGetConfigMapFunction(configMapName -> Optional.ofNullable(configMapStore.get(configMapName)))
+				.setCheckAndUpdateConfigMapFunction((configMapName, updateFunction) -> {
+					final KubernetesConfigMap configMap = configMapStore.get(configMapName);
+					if (configMap != null) {
+						try {
+							final boolean updated = updateFunction.apply(configMap).map(
+								updateConfigMap -> {
+									configMapStore.put(configMap.getName(), updateConfigMap);

Review comment:
        This is a testing kube client for contract unit test. Concurrent modification does not happen in such situation. So we do not need to retry here.
   
   For the concurrent tests, we need to use a real K8s cluster and client.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513460563



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/itcases/KubernetesITTestBase.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.itcases;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.kubeclient.DefaultKubeClientFactory;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubeClientFactory;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.StringUtils;
+
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for kubernetes integration tests with a configured real Kubernetes cluster and client. All the
+ * ITCases assume that the environment ITCASE_KUBECONFIG is set with a valid kube config file. In the E2E tests, we
+ * will use a minikube for the testing.
+ */
+public class KubernetesITTestBase {

Review comment:
       The functionality this class contains looks like a good candidate for an `ExternalResource`. That way we could include the `KubernetesResource` using `@ClassRule` or so into an IT case which needs it.

##########
File path: flink-end-to-end-tests/test-scripts/test_kubernetes_itcases.sh
##########
@@ -0,0 +1,30 @@
+#!/usr/bin/env bash
+################################################################################
+# 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.
+################################################################################
+
+source "$(dirname "$0")"/common_kubernetes.sh
+
+start_kubernetes
+
+# Set the ITCASE_KUBECONFIG environment since it is required to run the ITCases
+export ITCASE_KUBECONFIG=~/.kube/config
+
+cd $END_TO_END_DIR/../flink-kubernetes

Review comment:
       As a potential follow up, we could think about adding the Kubernetes resource as an `ExternalResource` which one can simply include in ones ITCase similar to `LocalStandaloneKafkaResource`. That way we could write end-to-end tests which can be simply run by using maven (similar to `SQLClientKafkaITCase`).

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();

Review comment:
       nit: We are checking that `kubeClient` and `ioExecutor` is not null but we don't do this check for `leaderConfig`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {

Review comment:
       maybe add an assertion that we are still running.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// 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) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not update ConfigMap " + configMapName, e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		final Optional<KubernetesConfigMap> configMapOpt = kubeClient.getConfigMap(configMapName);
+		if (configMapOpt.isPresent()) {
+			return KubernetesLeaderElector.hasLeadership(configMapOpt.get(), lockIdentity);
+		} else {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("ConfigMap " + configMapName + "does not exist.", null));
+			return false;
+		}
+	}
+
+	private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+		@Override
+		public void isLeader() {
+			synchronized (lock) {
+				if (running) {
+					leaderElectionEventHandler.onGrantLeadership();

Review comment:
       triggering callbacks under `lock` can cause deadlocks because the implementation might try to obtain the outer lock. If now at the same time the user of this class wants to close this component, then user will try to acquire the inner `lock` and we have a deadlock.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// 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) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not update ConfigMap " + configMapName, e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		final Optional<KubernetesConfigMap> configMapOpt = kubeClient.getConfigMap(configMapName);
+		if (configMapOpt.isPresent()) {
+			return KubernetesLeaderElector.hasLeadership(configMapOpt.get(), lockIdentity);
+		} else {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("ConfigMap " + configMapName + "does not exist.", null));
+			return false;
+		}
+	}
+
+	private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+		@Override
+		public void isLeader() {
+			synchronized (lock) {
+				if (running) {
+					leaderElectionEventHandler.onGrantLeadership();
+				}
+			}
+		}
+
+		@Override
+		public void notLeader() {
+			synchronized (lock) {
+				if (running) {
+					// Clear the leader information in ConfigMap
+					try {
+						kubeClient.checkAndUpdateConfigMap(
+							configMapName,
+							configMap -> {
+								if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+									configMap.getData().remove(LEADER_ADDRESS_KEY);
+									configMap.getData().remove(LEADER_SESSION_ID_KEY);
+									return Optional.of(configMap);
+								}
+								return Optional.empty();
+							}
+						).get();
+					} catch (Exception e) {
+						fatalErrorHandler.onFatalError(
+							new LeaderElectionException(
+								"Could not remove leader information from ConfigMap " + configMapName, e));
+					}
+					leaderElectionEventHandler.onRevokeLeadership();

Review comment:
       Same here with the potential deadlock problem.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -154,13 +231,68 @@ public Builder setWatchPodsAndDoCallbackFunction(BiFunction<Map<String, String>,
 			return this;
 		}
 
+		public Builder setCreateConfigMapFunction(
+				Function<KubernetesConfigMap, CompletableFuture<Void>> createConfigMapFunction) {
+			this.createConfigMapFunction = createConfigMapFunction;
+			return this;
+		}
+
+		public Builder setGetConfigMapFunction(Function<String, Optional<KubernetesConfigMap>> getConfigMapFunction) {
+			this.getConfigMapFunction = getConfigMapFunction;
+			return this;
+		}
+
+		public Builder setCheckAndUpdateConfigMapFunction(
+				BiFunction<String, Function<KubernetesConfigMap, Optional<KubernetesConfigMap>>,
+				CompletableFuture<Boolean>> checkAndUpdateConfigMapFunction) {

Review comment:
       nit: line breaks seem a bit odd.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/itcases/KubernetesLeaderElectionITCase.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.itcases;
+
+import org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionDriver;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.TestingContender;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+
+import org.junit.Test;
+
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_CONFIGMAP_NAME;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_URL;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.createLeaderElectionService;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.createLeaderRetrievalService;
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for the {@link DefaultLeaderElectionService} with {@link KubernetesLeaderElectionDriver} and
+ * {@link DefaultLeaderRetrievalService} with {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderRetrievalDriver}.
+ */
+public class KubernetesLeaderElectionITCase extends KubernetesITTestBase {

Review comment:
       I don't think that we have to test the `DefaultLeaderElectionService` against `K8s`. It should be enough to test the `KubernetesLeaderElectionDriver` or even only the `KubernetesLeaderElector`.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Base class for high availability unit tests with a configured testing Kubernetes client.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+	private static final String CLUSTER_ID = "leader-test-cluster";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	public static final String LEADER_URL = "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher";
+	public static final String LEADER_CONFIGMAP_NAME = "leader-test-cluster";
+
+	protected static final long TIMEOUT = 30L * 1000L;
+
+	protected ExecutorService executorService;
+	protected Configuration configuration;
+
+	@Before
+	public void setup() {
+		configuration = new Configuration();
+		configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
+		executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	}
+
+	@After
+	public void teardown() throws Exception {
+		executorService.shutdownNow();
+		executorService.awaitTermination(TIMEOUT, TimeUnit.MILLISECONDS);
+	}
+
+	/**
+	 * Context to leader election and retrieval tests.
+	 */
+	protected class Context {
+		/**
+		 * The configMapStore should only be used for set the following functions. Use a ConcurrentHashMap here since it
+		 * could be updated by {@link TestingFlinkKubeClient.MockKubernetesLeaderElector} and
+		 * {@link KubernetesLeaderElectionDriver}.
+		 */
+		private final Map<String, KubernetesConfigMap> configMapStore = new ConcurrentHashMap<>();
+
+		final AtomicBoolean leaderController = new AtomicBoolean(false);
+
+		final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+			configMapCallbackFutures = new ArrayList<>();
+
+		final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture = new CompletableFuture<>();
+		final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+		final FlinkKubeClient flinkKubeClient;
+
+		final LeaderElectionDriver leaderElectionDriver;
+		final TestingLeaderElectionEventHandler electionEventHandler;
+
+		final LeaderRetrievalDriver leaderRetrievalDriver;
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler;
+
+		Context() {
+			flinkKubeClient = getFlinkKubeClient();
+
+			electionEventHandler = new TestingLeaderElectionEventHandler(LEADER_URL);
+			leaderElectionDriver = createLeaderElectionDriver();
+
+			retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+			leaderRetrievalDriver = createLeaderRetrievalDriver();
+		}
+
+		void runTestAndGrantLeadershipToContender(RunnableWithException testMethod) throws Exception {
+			// Grant leadership
+			leaderController.set(true);
+			electionEventHandler.waitForLeader(TIMEOUT);
+			assertThat(electionEventHandler.isLeader(), is(true));
+
+			testMethod.run();
+
+			leaderElectionDriver.close();
+			leaderRetrievalDriver.close();
+		}
+
+		KubernetesConfigMap getLeaderConfigMap() {
+			final Optional<KubernetesConfigMap> configMapOpt = flinkKubeClient.getConfigMap(LEADER_CONFIGMAP_NAME);
+			assertThat(configMapOpt.isPresent(), is(true));
+			return configMapOpt.get();
+		}
+
+		protected FlinkKubeClient getFlinkKubeClient() {
+			return TestingFlinkKubeClient.builder()
+				.setCreateConfigMapFunction(configMap -> {
+					configMapStore.put(configMap.getName(), configMap);
+					return CompletableFuture.completedFuture(null);
+				})
+				.setGetConfigMapFunction(configMapName -> Optional.ofNullable(configMapStore.get(configMapName)))
+				.setCheckAndUpdateConfigMapFunction((configMapName, updateFunction) -> {
+					final KubernetesConfigMap configMap = configMapStore.get(configMapName);
+					if (configMap != null) {
+						try {
+							final boolean updated = updateFunction.apply(configMap).map(
+								updateConfigMap -> {
+									configMapStore.put(configMap.getName(), updateConfigMap);

Review comment:
       I guess we are not testing concurrent updates with this implementation because it simply overwrites the old information unlike the real implementation which would fail and then retry (with potentially incorporating the new information).

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Base class for high availability unit tests with a configured testing Kubernetes client.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+	private static final String CLUSTER_ID = "leader-test-cluster";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	public static final String LEADER_URL = "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher";
+	public static final String LEADER_CONFIGMAP_NAME = "leader-test-cluster";
+
+	protected static final long TIMEOUT = 30L * 1000L;
+
+	protected ExecutorService executorService;
+	protected Configuration configuration;
+
+	@Before
+	public void setup() {
+		configuration = new Configuration();
+		configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
+		executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	}
+
+	@After
+	public void teardown() throws Exception {
+		executorService.shutdownNow();
+		executorService.awaitTermination(TIMEOUT, TimeUnit.MILLISECONDS);
+	}
+
+	/**
+	 * Context to leader election and retrieval tests.
+	 */
+	protected class Context {
+		/**
+		 * The configMapStore should only be used for set the following functions. Use a ConcurrentHashMap here since it
+		 * could be updated by {@link TestingFlinkKubeClient.MockKubernetesLeaderElector} and
+		 * {@link KubernetesLeaderElectionDriver}.
+		 */
+		private final Map<String, KubernetesConfigMap> configMapStore = new ConcurrentHashMap<>();
+
+		final AtomicBoolean leaderController = new AtomicBoolean(false);
+
+		final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+			configMapCallbackFutures = new ArrayList<>();
+
+		final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture = new CompletableFuture<>();
+		final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+		final FlinkKubeClient flinkKubeClient;
+
+		final LeaderElectionDriver leaderElectionDriver;
+		final TestingLeaderElectionEventHandler electionEventHandler;
+
+		final LeaderRetrievalDriver leaderRetrievalDriver;
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler;
+
+		Context() {
+			flinkKubeClient = getFlinkKubeClient();
+
+			electionEventHandler = new TestingLeaderElectionEventHandler(LEADER_URL);
+			leaderElectionDriver = createLeaderElectionDriver();
+
+			retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+			leaderRetrievalDriver = createLeaderRetrievalDriver();
+		}
+
+		void runTestAndGrantLeadershipToContender(RunnableWithException testMethod) throws Exception {
+			// Grant leadership
+			leaderController.set(true);
+			electionEventHandler.waitForLeader(TIMEOUT);
+			assertThat(electionEventHandler.isLeader(), is(true));
+
+			testMethod.run();
+
+			leaderElectionDriver.close();
+			leaderRetrievalDriver.close();
+		}
+
+		KubernetesConfigMap getLeaderConfigMap() {
+			final Optional<KubernetesConfigMap> configMapOpt = flinkKubeClient.getConfigMap(LEADER_CONFIGMAP_NAME);
+			assertThat(configMapOpt.isPresent(), is(true));
+			return configMapOpt.get();
+		}
+
+		protected FlinkKubeClient getFlinkKubeClient() {
+			return TestingFlinkKubeClient.builder()
+				.setCreateConfigMapFunction(configMap -> {
+					configMapStore.put(configMap.getName(), configMap);
+					return CompletableFuture.completedFuture(null);
+				})
+				.setGetConfigMapFunction(configMapName -> Optional.ofNullable(configMapStore.get(configMapName)))
+				.setCheckAndUpdateConfigMapFunction((configMapName, updateFunction) -> {
+					final KubernetesConfigMap configMap = configMapStore.get(configMapName);
+					if (configMap != null) {
+						try {
+							final boolean updated = updateFunction.apply(configMap).map(
+								updateConfigMap -> {
+									configMapStore.put(configMap.getName(), updateConfigMap);
+									return true;
+								}).orElse(false);
+							return CompletableFuture.completedFuture(updated);
+						} catch (Throwable throwable) {
+							throw new CompletionException(new KubernetesException("Failed to execute update fuction"));
+						}
+					}
+					throw new CompletionException(
+						new KubernetesException("ConfigMap " + configMapName + " does not exists."));
+				})
+				.setWatchConfigMapsFunction((ignore, handler) -> {
+					final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> future =
+						new CompletableFuture<>();
+					future.complete(handler);

Review comment:
       could be `CompletableFuture.completedFuture`

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Base class for high availability unit tests with a configured testing Kubernetes client.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+	private static final String CLUSTER_ID = "leader-test-cluster";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	public static final String LEADER_URL = "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher";
+	public static final String LEADER_CONFIGMAP_NAME = "leader-test-cluster";
+
+	protected static final long TIMEOUT = 30L * 1000L;
+
+	protected ExecutorService executorService;
+	protected Configuration configuration;
+
+	@Before
+	public void setup() {
+		configuration = new Configuration();
+		configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
+		executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	}
+
+	@After
+	public void teardown() throws Exception {
+		executorService.shutdownNow();
+		executorService.awaitTermination(TIMEOUT, TimeUnit.MILLISECONDS);
+	}
+
+	/**
+	 * Context to leader election and retrieval tests.
+	 */
+	protected class Context {
+		/**
+		 * The configMapStore should only be used for set the following functions. Use a ConcurrentHashMap here since it
+		 * could be updated by {@link TestingFlinkKubeClient.MockKubernetesLeaderElector} and
+		 * {@link KubernetesLeaderElectionDriver}.
+		 */
+		private final Map<String, KubernetesConfigMap> configMapStore = new ConcurrentHashMap<>();
+
+		final AtomicBoolean leaderController = new AtomicBoolean(false);
+
+		final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+			configMapCallbackFutures = new ArrayList<>();
+
+		final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture = new CompletableFuture<>();
+		final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+		final FlinkKubeClient flinkKubeClient;
+
+		final LeaderElectionDriver leaderElectionDriver;
+		final TestingLeaderElectionEventHandler electionEventHandler;
+
+		final LeaderRetrievalDriver leaderRetrievalDriver;
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler;
+
+		Context() {
+			flinkKubeClient = getFlinkKubeClient();
+
+			electionEventHandler = new TestingLeaderElectionEventHandler(LEADER_URL);
+			leaderElectionDriver = createLeaderElectionDriver();
+
+			retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+			leaderRetrievalDriver = createLeaderRetrievalDriver();
+		}
+
+		void runTestAndGrantLeadershipToContender(RunnableWithException testMethod) throws Exception {
+			// Grant leadership
+			leaderController.set(true);
+			electionEventHandler.waitForLeader(TIMEOUT);
+			assertThat(electionEventHandler.isLeader(), is(true));
+
+			testMethod.run();
+
+			leaderElectionDriver.close();
+			leaderRetrievalDriver.close();
+		}
+
+		KubernetesConfigMap getLeaderConfigMap() {
+			final Optional<KubernetesConfigMap> configMapOpt = flinkKubeClient.getConfigMap(LEADER_CONFIGMAP_NAME);
+			assertThat(configMapOpt.isPresent(), is(true));
+			return configMapOpt.get();
+		}
+
+		protected FlinkKubeClient getFlinkKubeClient() {
+			return TestingFlinkKubeClient.builder()
+				.setCreateConfigMapFunction(configMap -> {
+					configMapStore.put(configMap.getName(), configMap);
+					return CompletableFuture.completedFuture(null);
+				})
+				.setGetConfigMapFunction(configMapName -> Optional.ofNullable(configMapStore.get(configMapName)))
+				.setCheckAndUpdateConfigMapFunction((configMapName, updateFunction) -> {
+					final KubernetesConfigMap configMap = configMapStore.get(configMapName);
+					if (configMap != null) {
+						try {
+							final boolean updated = updateFunction.apply(configMap).map(
+								updateConfigMap -> {
+									configMapStore.put(configMap.getName(), updateConfigMap);
+									return true;
+								}).orElse(false);
+							return CompletableFuture.completedFuture(updated);
+						} catch (Throwable throwable) {
+							throw new CompletionException(new KubernetesException("Failed to execute update fuction"));
+						}
+					}
+					throw new CompletionException(
+						new KubernetesException("ConfigMap " + configMapName + " does not exists."));
+				})
+				.setWatchConfigMapsFunction((ignore, handler) -> {
+					final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> future =
+						new CompletableFuture<>();
+					future.complete(handler);
+					configMapCallbackFutures.add(future);
+					return new TestingFlinkKubeClient.MockKubernetesWatch();
+				})
+				.setDeleteConfigMapFunction(name -> {
+					configMapStore.remove(name);
+					return FutureUtils.completedVoidFuture();
+				})
+				.setDeleteConfigMapByLabelFunction(labels -> {
+					deleteConfigMapByLabelsFuture.complete(labels);
+					return FutureUtils.completedVoidFuture();
+				})
+				.setCloseConsumer(closeKubeClientFuture::complete)
+				.setLeaderController(leaderController)
+				.build();
+		}
+
+		private LeaderElectionDriver createLeaderElectionDriver() {
+			final KubernetesLeaderElectionConfiguration leaderConfig = new KubernetesLeaderElectionConfiguration(
+				LEADER_CONFIGMAP_NAME, LOCK_IDENTITY, configuration);
+			final KubernetesLeaderElectionDriverFactory factory = new KubernetesLeaderElectionDriverFactory(
+				flinkKubeClient, executorService, leaderConfig);
+			return factory.createLeaderElectionDriver(
+				electionEventHandler, electionEventHandler::handleError, LEADER_URL);
+		}
+
+		private LeaderRetrievalDriver createLeaderRetrievalDriver() {
+			final KubernetesLeaderRetrievalDriverFactory factory = new KubernetesLeaderRetrievalDriverFactory(
+				flinkKubeClient, LEADER_CONFIGMAP_NAME);
+			return factory.createLeaderRetrievalDriver(retrievalEventHandler, retrievalEventHandler::handleError);
+		}
+	}
+
+	public static DefaultLeaderElectionService createLeaderElectionService(
+			Configuration configuration,
+			FlinkKubeClient kubeClient,
+			String configMapName,
+			String lockIdentity,
+			ExecutorService executorService) {
+		final KubernetesLeaderElectionConfiguration leaderConfig = new KubernetesLeaderElectionConfiguration(
+			configMapName, lockIdentity, configuration);
+		return new DefaultLeaderElectionService(new KubernetesLeaderElectionDriverFactory(
+			kubeClient, executorService, leaderConfig));
+	}
+
+	public static DefaultLeaderRetrievalService createLeaderRetrievalService(
+			FlinkKubeClient flinkKubeClient, String configMapName) {
+		return new DefaultLeaderRetrievalService(
+			new KubernetesLeaderRetrievalDriverFactory(flinkKubeClient, configMapName));
+	}

Review comment:
       Why are these methods contained in this class? It is a bit confusing which base is responsible for what, I have to admit.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);

Review comment:
       Not sure, but can't it happen that we are missing leadership events if `leaderElector::run` notifes `LeaderCallbackHandlerImpl` before `running` is set to `true`?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// 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) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not update ConfigMap " + configMapName, e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		final Optional<KubernetesConfigMap> configMapOpt = kubeClient.getConfigMap(configMapName);
+		if (configMapOpt.isPresent()) {
+			return KubernetesLeaderElector.hasLeadership(configMapOpt.get(), lockIdentity);
+		} else {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("ConfigMap " + configMapName + "does not exist.", null));
+			return false;
+		}
+	}
+
+	private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+		@Override
+		public void isLeader() {
+			synchronized (lock) {
+				if (running) {
+					leaderElectionEventHandler.onGrantLeadership();
+				}
+			}
+		}
+
+		@Override
+		public void notLeader() {
+			synchronized (lock) {
+				if (running) {
+					// Clear the leader information in ConfigMap
+					try {
+						kubeClient.checkAndUpdateConfigMap(
+							configMapName,
+							configMap -> {
+								if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+									configMap.getData().remove(LEADER_ADDRESS_KEY);
+									configMap.getData().remove(LEADER_SESSION_ID_KEY);
+									return Optional.of(configMap);
+								}
+								return Optional.empty();
+							}

Review comment:
       I am wondering whether this isn't something the `DefaultLeaderElectionService` can decide when receiving the `onRevokeLeadership` callback. It could then decide to call `writeLeaderInformation(LeaderInformation.empty())`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);

Review comment:
       Why are we completing the `leaderElectorRunFuture` here? What is the purpose of `leaderElectorRunFuture`?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -217,6 +230,79 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.exceptionally(
+				throwable -> {
+					throw new CompletionException(
+						new KubernetesException("Failed to create ConfigMap " + configMapName, throwable));
+				});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			Function<KubernetesConfigMap, Optional<KubernetesConfigMap>> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(
+						configMap -> function.apply(configMap).map(
+							updatedConfigMap -> {
+								try {
+									this.internalClient.configMaps()
+										.inNamespace(namespace)
+										.withName(configMapName)
+										.lockResourceVersion(updatedConfigMap.getResourceVersion())
+										.replace(updatedConfigMap.getInternalResource());
+								} catch (Throwable throwable) {
+									LOG.debug("Failed to update ConfigMap {} with data {} because of concurrent " +
+										"modifications. Trying again.", configMap.getName(), configMap.getData());
+									throw throwable;
+								}
+								return true;
+							}).orElse(false))
+					.orElseThrow(() -> new CompletionException(
+						new KubernetesException("Not retry since " + configMapName + " does not exist."))),

Review comment:
       Maybe: `"Cannot retry checkAndUpdateConfigMap with configMap " + configMapName + " because it does not exist."`

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// Get the updated ConfigMap with new leader information
+						if (confirmedLeaderAddress != null && confirmedLeaderSessionID != null) {

Review comment:
       Does `leaderInformation` must always contain a leader address? So we are not allowing to explicitly remove the leader information?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);

Review comment:
       What is the contract of `leaderElector::run`? Could it be that it blocks a thread until it obtains leadership? Isn't this a bit dangerous if we have multiple leader contenders in the same process? Then we would block `n-1` threads of the `ioExecutor` indefinitely if `n` is the number of leader contenders. How large is `ioExecutor` and for what other purposes do we use it?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// 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) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not update ConfigMap " + configMapName, e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		final Optional<KubernetesConfigMap> configMapOpt = kubeClient.getConfigMap(configMapName);
+		if (configMapOpt.isPresent()) {
+			return KubernetesLeaderElector.hasLeadership(configMapOpt.get(), lockIdentity);
+		} else {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("ConfigMap " + configMapName + "does not exist.", null));
+			return false;
+		}
+	}
+
+	private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+		@Override
+		public void isLeader() {
+			synchronized (lock) {
+				if (running) {
+					leaderElectionEventHandler.onGrantLeadership();
+				}
+			}
+		}
+
+		@Override
+		public void notLeader() {
+			synchronized (lock) {
+				if (running) {
+					// Clear the leader information in ConfigMap
+					try {
+						kubeClient.checkAndUpdateConfigMap(
+							configMapName,
+							configMap -> {
+								if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+									configMap.getData().remove(LEADER_ADDRESS_KEY);
+									configMap.getData().remove(LEADER_SESSION_ID_KEY);
+									return Optional.of(configMap);
+								}
+								return Optional.empty();
+							}
+						).get();
+					} catch (Exception e) {
+						fatalErrorHandler.onFatalError(
+							new LeaderElectionException(
+								"Could not remove leader information from ConfigMap " + configMapName, e));
+					}
+					leaderElectionEventHandler.onRevokeLeadership();
+					// Continue to contend the leader
+					if (!leaderElectorRunFuture.isDone()) {
+						leaderElectorRunFuture.complete(null);
+					}

Review comment:
       What should completing this future do?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership

Review comment:
       nit: leading whitespace

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// 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) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not update ConfigMap " + configMapName, e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {

Review comment:
       Same here with the assertion that we are still running.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// 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) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not update ConfigMap " + configMapName, e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		final Optional<KubernetesConfigMap> configMapOpt = kubeClient.getConfigMap(configMapName);
+		if (configMapOpt.isPresent()) {
+			return KubernetesLeaderElector.hasLeadership(configMapOpt.get(), lockIdentity);
+		} else {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("ConfigMap " + configMapName + "does not exist.", null));
+			return false;
+		}
+	}
+
+	private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+		@Override
+		public void isLeader() {
+			synchronized (lock) {
+				if (running) {
+					leaderElectionEventHandler.onGrantLeadership();
+				}
+			}
+		}
+
+		@Override
+		public void notLeader() {
+			synchronized (lock) {
+				if (running) {
+					// Clear the leader information in ConfigMap
+					try {
+						kubeClient.checkAndUpdateConfigMap(
+							configMapName,
+							configMap -> {
+								if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+									configMap.getData().remove(LEADER_ADDRESS_KEY);
+									configMap.getData().remove(LEADER_SESSION_ID_KEY);
+									return Optional.of(configMap);
+								}
+								return Optional.empty();
+							}
+						).get();
+					} catch (Exception e) {
+						fatalErrorHandler.onFatalError(
+							new LeaderElectionException(
+								"Could not remove leader information from ConfigMap " + configMapName, e));
+					}
+					leaderElectionEventHandler.onRevokeLeadership();
+					// Continue to contend the leader
+					if (!leaderElectorRunFuture.isDone()) {
+						leaderElectorRunFuture.complete(null);
+					}
+					leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+					FutureUtils.assertNoException(leaderElectorRunFuture);
+				}
+			}
+		}
+	}
+
+	private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> {
+		// This is used to get the difference between current and previous data. Since the annotation will be updated
+		// periodically, we need to filter out these noises.
+		final Map<String, String> previousData = new HashMap<>();
+
+		@Override
+		public void onAdded(List<KubernetesConfigMap> configMaps) {
+			// noop
+		}
+
+		@Override
+		public void onModified(List<KubernetesConfigMap> configMaps) {
+			// We should only receive events for the watched ConfigMap
+			final KubernetesConfigMap configMap = checkConfigMaps(configMaps, configMapName);
+
+			if (!configMap.getData().equals(previousData)) {
+				if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+					synchronized (lock) {
+						if (running) {
+							leaderElectionEventHandler.onLeaderInformationChange(
+								getLeaderInformationFromConfigMap(configMap));

Review comment:
       Will we use a different configMap than this one for storing the completed checkpoint pointers? I no, then `onModified` could also been called because we added a new checkpoint pointer.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalTest.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 KubernetesLeaderRetrievalDriver}.
+ */
+public class KubernetesLeaderRetrievalTest extends KubernetesHighAvailabilityTestBase {
+
+	@Test
+	public void testKubernetesLeaderRetrievalOnAdded() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					leaderElectionDriver.writeLeaderInformation(electionEventHandler.getLeaderInformation());
+					assertThat(configMapCallbackFutures.size(), is(2));
+					final FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> callbackHandler =
+						configMapCallbackFutures.get(1).get(TIMEOUT, TimeUnit.MILLISECONDS);

Review comment:
       maybe we could introduce something like `getCallbackHandler` which encapsulates this logic.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriver.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.List;
+
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link KubernetesLeaderElectionDriver}.
+ * {@link LeaderRetrievalDriver} implementation for Kubernetes. It retrieves the current leader which has
+ * been elected by the {@link KubernetesLeaderElectionDriver}.
+ * The leader address as well as the current leader session ID is retrieved from Kubernetes ConfigMap.
+ */
+public class KubernetesLeaderRetrievalDriver implements LeaderRetrievalDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderRetrievalDriver.class);
+
+	private final Object lock = new Object();
+
+	private final String configMapName;
+
+	private final LeaderRetrievalEventHandler leaderRetrievalEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderRetrievalDriver(
+			FlinkKubeClient kubeClient,
+			String configMapName,
+			LeaderRetrievalEventHandler leaderRetrievalEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+		checkNotNull(kubeClient, "Kubernetes client");
+		this.configMapName = checkNotNull(configMapName, "ConfigMap name");
+		this.leaderRetrievalEventHandler = checkNotNull(leaderRetrievalEventHandler, "LeaderRetrievalEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Stopping {}.", this);
+			kubernetesWatch.close();
+		}
+	}
+
+	private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> {
+
+		@Override
+		public void onAdded(List<KubernetesConfigMap> configMaps) {
+			handleEvent(configMaps);
+		}
+
+		@Override
+		public void onModified(List<KubernetesConfigMap> configMaps) {
+			handleEvent(configMaps);
+		}
+
+		@Override
+		public void onDeleted(List<KubernetesConfigMap> configMaps) {
+			// Nothing to do since the delete event will be handled in the leader election part.
+		}
+
+		@Override
+		public void onError(List<KubernetesConfigMap> configMaps) {
+			synchronized (lock) {
+				if (running) {
+					fatalErrorHandler.onFatalError(new Exception("Error while watching the ConfigMap " + configMapName));
+				}
+			}
+		}
+
+		@Override
+		public void handleFatalError(Throwable throwable) {
+			synchronized (lock) {
+				if (running) {
+					fatalErrorHandler.onFatalError(new Exception("Error while watching the ConfigMap " + configMapName));
+				}
+			}
+		}
+
+		@GuardedBy("lock")
+		private void handleEvent(List<KubernetesConfigMap> configMaps) {
+			synchronized (lock) {
+				if (running) {
+					final KubernetesConfigMap configMap = checkConfigMaps(configMaps, configMapName);
+					leaderRetrievalEventHandler.notifyLeaderAddress(getLeaderInformationFromConfigMap(configMap));

Review comment:
       same here with the inner `lock`.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalTest.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 KubernetesLeaderRetrievalDriver}.
+ */
+public class KubernetesLeaderRetrievalTest extends KubernetesHighAvailabilityTestBase {

Review comment:
       Let's call test classes which test `X` `XTest`. Hence, here it should be called `KubernetesLeaderRetrievalDriverTest`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -125,6 +130,45 @@ public static String getDeploymentName(String clusterId) {
 		return commonLabels;
 	}
 
+	/**
+	 * Get ConfigMap labels for the current Flink cluster. They could be used to filter and clean-up the resources.
+	 *
+	 * @param clusterId cluster id

Review comment:
       `type` is missing

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriver.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.List;
+
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link KubernetesLeaderElectionDriver}.
+ * {@link LeaderRetrievalDriver} implementation for Kubernetes. It retrieves the current leader which has
+ * been elected by the {@link KubernetesLeaderElectionDriver}.
+ * The leader address as well as the current leader session ID is retrieved from Kubernetes ConfigMap.
+ */
+public class KubernetesLeaderRetrievalDriver implements LeaderRetrievalDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderRetrievalDriver.class);
+
+	private final Object lock = new Object();
+
+	private final String configMapName;
+
+	private final LeaderRetrievalEventHandler leaderRetrievalEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderRetrievalDriver(
+			FlinkKubeClient kubeClient,
+			String configMapName,
+			LeaderRetrievalEventHandler leaderRetrievalEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+		checkNotNull(kubeClient, "Kubernetes client");
+		this.configMapName = checkNotNull(configMapName, "ConfigMap name");
+		this.leaderRetrievalEventHandler = checkNotNull(leaderRetrievalEventHandler, "LeaderRetrievalEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Stopping {}.", this);
+			kubernetesWatch.close();
+		}
+	}
+
+	private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> {
+
+		@Override
+		public void onAdded(List<KubernetesConfigMap> configMaps) {
+			handleEvent(configMaps);
+		}
+
+		@Override
+		public void onModified(List<KubernetesConfigMap> configMaps) {
+			handleEvent(configMaps);
+		}
+
+		@Override
+		public void onDeleted(List<KubernetesConfigMap> configMaps) {
+			// Nothing to do since the delete event will be handled in the leader election part.
+		}
+
+		@Override
+		public void onError(List<KubernetesConfigMap> configMaps) {
+			synchronized (lock) {
+				if (running) {
+					fatalErrorHandler.onFatalError(new Exception("Error while watching the ConfigMap " + configMapName));

Review comment:
       Dangerous to call callbacks under the inner `lock`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	private final String leaderContenderDescription;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler Event handler for processing leader change events
+	 * @param fatalErrorHandler Fatal error handler
+	 * @param leaderContenderDescription Leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}", this);
+
+			client.getUnhandledErrorListenable().removeListener(this);
+
+			client.getConnectionStateListenable().removeListener(listener);
+
+			Exception exception = null;
+
+			try {
+				cache.close();
+			} catch (Exception e) {
+				exception = e;
+			}
+
+			try {
+				leaderLatch.close();
+			} catch (Exception e) {
+				exception = ExceptionUtils.firstOrSuppressed(e, exception);
+			}
+
+			if (exception != null) {
+				throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		synchronized (lock) {
+			if (running) {
+				leaderElectionEventHandler.onGrantLeadership();
+			}
+		}

Review comment:
       I think I might have given you bad advice here. Executing the callbacks under the inner `lock` is problematic if there is an outer call which holds the outer lock and calls into this class trying to obtain the inner `lock` (e.g. when concurrently closing this instance). Maybe we have to say in the contract of `ZooKeeperLeaderElectionDriver `, that spurious callbacks to `onGrantLeadership` can happen even after the driver is stopped and it is the responsibility of the implementor to filter invalid calls out (e.g. after close has been called on the outer class).

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/itcases/KubernetesLeaderElectionITCase.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.itcases;
+
+import org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionDriver;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.TestingContender;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+
+import org.junit.Test;
+
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_CONFIGMAP_NAME;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_URL;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.createLeaderElectionService;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.createLeaderRetrievalService;
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for the {@link DefaultLeaderElectionService} with {@link KubernetesLeaderElectionDriver} and
+ * {@link DefaultLeaderRetrievalService} with {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderRetrievalDriver}.
+ */
+public class KubernetesLeaderElectionITCase extends KubernetesITTestBase {
+
+	/**
+	 * Start multiple leaders concurrently, if current leader dies, a new one could take over and update the leader
+	 * information successfully. The leader retrieval service should notice this change and notify listener with new
+	 * leader address.
+	 */
+	@Test
+	public void testConcurrentLeaderElectionAndRetrieval() throws Exception {
+		final String leaderConfigMapName = LEADER_CONFIGMAP_NAME + System.currentTimeMillis();
+		final int leaderNum = 3;
+		final DefaultLeaderElectionService[] leaderElectionServices = new DefaultLeaderElectionService[leaderNum];
+		final ExecutorService[] executorServices = new ExecutorService[leaderNum];
+		DefaultLeaderRetrievalService leaderRetrievalService = null;
+
+		try {
+			// Start multiple leader contenders
+			final TestingContender[] contenders = new TestingContender[leaderNum];
+			final String[] lockIdentities = new String[leaderNum];
+			for (int i = 0; i < leaderNum; i++) {
+				final FlinkKubeClient kubeClientForLeader = kubeClientFactory.fromConfiguration(configuration);
+				lockIdentities[i] = UUID.randomUUID().toString();
+				executorServices[i] = Executors.newFixedThreadPool(
+					4, new ExecutorThreadFactory("IO-Executor-Leader-" + i));
+				leaderElectionServices[i] = createLeaderElectionService(
+					configuration, kubeClientForLeader, leaderConfigMapName, lockIdentities[i], executorServices[i]);
+				contenders[i] = new TestingContender(getLeaderUrl(i), leaderElectionServices[i]);
+				leaderElectionServices[i].start(contenders[i]);
+			}
+
+			// Start the leader retrieval
+			leaderRetrievalService = createLeaderRetrievalService(flinkKubeClient, leaderConfigMapName);
+			final TestingListener listener = new TestingListener();
+			leaderRetrievalService.start(listener);
+
+			final String currentLeaderAddress = listener.waitForNewLeader(TIMEOUT);
+			final String currentLeaderSessionId = listener.getLeaderSessionID().toString();
+			final Optional<KubernetesConfigMap> configMapOpt = flinkKubeClient.getConfigMap(leaderConfigMapName);
+			assertThat(configMapOpt.isPresent(), is(true));
+			final String currentLock = configMapOpt.get().getAnnotations().get(LEADER_ANNOTATION_KEY);
+			for (int i = 0; i < 3; i++) {

Review comment:
       `i < leaderNum`

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_KEY;
+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.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link DefaultLeaderElectionService} with {@link KubernetesLeaderElectionDriver}.
+ */
+public class KubernetesLeaderElectionTest extends KubernetesHighAvailabilityTestBase {
+
+	@Test
+	public void testKubernetesLeaderElection() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					assertThat(electionEventHandler.getLeaderInformation().getLeaderAddress(), is(LEADER_URL));
+
+					// Revoke leader
+					leaderController.set(false);
+					electionEventHandler.waitForRevokeLeader(TIMEOUT);
+					assertThat(electionEventHandler.getLeaderInformation(), is(LeaderInformation.empty()));
+				});
+		}};

Review comment:
       What I don't really like about this test is that one does not really see what's going on. The test start right away with an assertion which is only clear when digging into the `Context`. I think it is better to write tests so that one sees w/o a lot of difficulties what's being tested.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/itcases/KubernetesLeaderElectionITCase.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.itcases;
+
+import org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionDriver;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.TestingContender;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+
+import org.junit.Test;
+
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_CONFIGMAP_NAME;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.LEADER_URL;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.createLeaderElectionService;
+import static org.apache.flink.kubernetes.highavailability.KubernetesHighAvailabilityTestBase.createLeaderRetrievalService;
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for the {@link DefaultLeaderElectionService} with {@link KubernetesLeaderElectionDriver} and
+ * {@link DefaultLeaderRetrievalService} with {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderRetrievalDriver}.
+ */
+public class KubernetesLeaderElectionITCase extends KubernetesITTestBase {
+
+	/**
+	 * Start multiple leaders concurrently, if current leader dies, a new one could take over and update the leader
+	 * information successfully. The leader retrieval service should notice this change and notify listener with new
+	 * leader address.
+	 */
+	@Test
+	public void testConcurrentLeaderElectionAndRetrieval() throws Exception {

Review comment:
       I think this test could be more targeted by simply testing multiple `KubernetesLeaderElector` instead of the `DefaultLeaderElectionService`.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Base class for high availability unit tests with a configured testing Kubernetes client.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+	private static final String CLUSTER_ID = "leader-test-cluster";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	public static final String LEADER_URL = "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher";
+	public static final String LEADER_CONFIGMAP_NAME = "leader-test-cluster";
+
+	protected static final long TIMEOUT = 30L * 1000L;
+
+	protected ExecutorService executorService;
+	protected Configuration configuration;
+
+	@Before
+	public void setup() {
+		configuration = new Configuration();
+		configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
+		executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	}
+
+	@After
+	public void teardown() throws Exception {
+		executorService.shutdownNow();
+		executorService.awaitTermination(TIMEOUT, TimeUnit.MILLISECONDS);
+	}
+
+	/**
+	 * Context to leader election and retrieval tests.
+	 */
+	protected class Context {
+		/**
+		 * The configMapStore should only be used for set the following functions. Use a ConcurrentHashMap here since it
+		 * could be updated by {@link TestingFlinkKubeClient.MockKubernetesLeaderElector} and
+		 * {@link KubernetesLeaderElectionDriver}.
+		 */
+		private final Map<String, KubernetesConfigMap> configMapStore = new ConcurrentHashMap<>();
+
+		final AtomicBoolean leaderController = new AtomicBoolean(false);
+
+		final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+			configMapCallbackFutures = new ArrayList<>();
+
+		final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture = new CompletableFuture<>();
+		final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+		final FlinkKubeClient flinkKubeClient;
+
+		final LeaderElectionDriver leaderElectionDriver;
+		final TestingLeaderElectionEventHandler electionEventHandler;
+
+		final LeaderRetrievalDriver leaderRetrievalDriver;
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler;
+
+		Context() {
+			flinkKubeClient = getFlinkKubeClient();
+
+			electionEventHandler = new TestingLeaderElectionEventHandler(LEADER_URL);
+			leaderElectionDriver = createLeaderElectionDriver();
+
+			retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+			leaderRetrievalDriver = createLeaderRetrievalDriver();
+		}
+
+		void runTestAndGrantLeadershipToContender(RunnableWithException testMethod) throws Exception {
+			// Grant leadership
+			leaderController.set(true);
+			electionEventHandler.waitForLeader(TIMEOUT);
+			assertThat(electionEventHandler.isLeader(), is(true));
+
+			testMethod.run();
+
+			leaderElectionDriver.close();
+			leaderRetrievalDriver.close();
+		}
+
+		KubernetesConfigMap getLeaderConfigMap() {
+			final Optional<KubernetesConfigMap> configMapOpt = flinkKubeClient.getConfigMap(LEADER_CONFIGMAP_NAME);
+			assertThat(configMapOpt.isPresent(), is(true));
+			return configMapOpt.get();
+		}
+
+		protected FlinkKubeClient getFlinkKubeClient() {
+			return TestingFlinkKubeClient.builder()
+				.setCreateConfigMapFunction(configMap -> {
+					configMapStore.put(configMap.getName(), configMap);
+					return CompletableFuture.completedFuture(null);
+				})
+				.setGetConfigMapFunction(configMapName -> Optional.ofNullable(configMapStore.get(configMapName)))
+				.setCheckAndUpdateConfigMapFunction((configMapName, updateFunction) -> {
+					final KubernetesConfigMap configMap = configMapStore.get(configMapName);
+					if (configMap != null) {
+						try {
+							final boolean updated = updateFunction.apply(configMap).map(
+								updateConfigMap -> {
+									configMapStore.put(configMap.getName(), updateConfigMap);
+									return true;
+								}).orElse(false);
+							return CompletableFuture.completedFuture(updated);
+						} catch (Throwable throwable) {
+							throw new CompletionException(new KubernetesException("Failed to execute update fuction"));
+						}
+					}
+					throw new CompletionException(
+						new KubernetesException("ConfigMap " + configMapName + " does not exists."));
+				})
+				.setWatchConfigMapsFunction((ignore, handler) -> {
+					final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> future =
+						new CompletableFuture<>();
+					future.complete(handler);
+					configMapCallbackFutures.add(future);
+					return new TestingFlinkKubeClient.MockKubernetesWatch();
+				})
+				.setDeleteConfigMapFunction(name -> {
+					configMapStore.remove(name);
+					return FutureUtils.completedVoidFuture();
+				})
+				.setDeleteConfigMapByLabelFunction(labels -> {
+					deleteConfigMapByLabelsFuture.complete(labels);

Review comment:
       Can't this method be called multiple times? We would only complete the future for the first call, right?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Base class for high availability unit tests with a configured testing Kubernetes client.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {

Review comment:
       The complexity of this class somehow gives me the feeling that the `KubernetesLeaderElectionDriver` is not really easy to test. Moreover, we might try to squeeze too many things into this class which adds to this complexity. For example, instead of implementing a `MockKubernetesLeaderElector` wouldn't it be enough to have some testing utility which captures the `LeaderCallbackHandler` on which the test could do calls?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_KEY;
+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.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link DefaultLeaderElectionService} with {@link KubernetesLeaderElectionDriver}.

Review comment:
       It is confusing that this comment says that we are testing the `DefaultLeaderElectionService` here. I don't see it in the 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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509296824



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionServiceTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.apache.flink.runtime.leaderelection.TestingContender;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_KEY;
+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 KubernetesLeaderElectionService}.
+ */
+public class KubernetesLeaderElectionServiceTest extends KubernetesHighAvailabilityTestBase {
+
+	@Test
+	public void testKubernetesLeaderElection() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					assertThat(leaderElectionService.getLeaderSessionID(), is(contender.getLeaderSessionID()));
+					assertThat(configMapStore.size(), is(1));
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+
+					// Revoke leader
+					leaderController.set(false);
+					contender.waitForRevokeLeader(TIMEOUT);
+					assertThat(leaderElectionService.getLeaderSessionID(), nullValue());
+					assertThat(getLeaderConfigMap().getData().size(), is(0));
+				});
+		}};
+	}
+
+	@Test
+	public void testLeaderConfigMapDeletedExternally() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					final FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> callbackHandler =
+						configMapsAndDoCallbackFuture.get(TIMEOUT, TimeUnit.MILLISECONDS);
+					assertThat(callbackHandler, is(notNullValue()));
+
+					callbackHandler.onDeleted(Collections.singletonList(configMapStore.remove(LEADER_CONFIGMAP_NAME)));
+					// The ConfigMap should be created again.
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+				});
+		}};
+	}
+
+	@Test
+	public void testLeaderConfigMapUpdatedExternally() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					final FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> callbackHandler =
+						configMapsAndDoCallbackFuture.get(TIMEOUT, TimeUnit.MILLISECONDS);
+					assertThat(callbackHandler, is(notNullValue()));
+					// Update ConfigMap with wrong data
+					getLeaderConfigMap().getData().put(Constants.LEADER_ADDRESS_KEY, "wrong data");
+					callbackHandler.onModified(Collections.singletonList(configMapStore.get(LEADER_CONFIGMAP_NAME)));
+					// The ConfigMap should be corrected
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+				});
+		}};
+	}
+
+	/**
+	 * Start multiple leaders, if current leader dies, a new one could take over and update the leader
+	 * information successfully.
+	 */
+	@Test
+	public void testMultipleLeaders() throws Exception {
+		final int leaderNum = 3;
+		final AtomicBoolean[] leaderController = new AtomicBoolean[leaderNum];
+		final TestingContender[] contenders = new TestingContender[leaderNum];
+		final KubernetesLeaderElectionService[] leaderElectionServices = new KubernetesLeaderElectionService[leaderNum];
+		for (int i = 0; i < leaderNum; i++) {
+			leaderController[i] = new AtomicBoolean(false);
+			leaderElectionServices[i] = new Context().createLeaderElectionService(leaderController[i]);
+			contenders[i] = new TestingContender(getLeaderUrl(i), leaderElectionServices[i]);
+			leaderElectionServices[i].start(contenders[i]);
+		}
+		leaderController[0].set(true);
+		contenders[0].waitForLeader(TIMEOUT);
+		assertThat(
+			configMapStore.get(LEADER_CONFIGMAP_NAME).getData().get(Constants.LEADER_ADDRESS_KEY),
+			is(getLeaderUrl(0)));
+		// Leader 0 died
+		leaderController[0].set(false);
+		contenders[0].waitForRevokeLeader(TIMEOUT);
+		// Leader 2 try to acquire
+		leaderController[2].set(true);
+		contenders[2].waitForLeader(TIMEOUT);
+		assertThat(
+			configMapStore.get(LEADER_CONFIGMAP_NAME).getData().get(Constants.LEADER_ADDRESS_KEY),
+			is(getLeaderUrl(2)));
+		for (int i = 0; i < leaderNum; i++) {
+			leaderElectionServices[i].stop();
+		}
+	}

Review comment:
       It is hard to use the real `KubernetesLeaderElector` in the UT. That's why I add a `MockKubernetesLeaderElector` in the `TestingFlinkKubeClient` to valid the contract tests for leader election/retrieval.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e33c52394024e8f1312c096a4b612d53d8848b97 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653) 
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 991bdfa5427479f845abaf91caba0069011ee9e3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516518855



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.DefaultKubeClientFactory;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubeClientFactory;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for the {@link KubernetesLeaderElector}. Start multiple leader contenders currently, one should elect
+ * successfully. And if current leader dies, a new one could take over.
+ */
+public class KubernetesLeaderElectorITCase {
+
+	@ClassRule
+	public static KubernetesResource kubernetesResource = new KubernetesResource();
+
+	private static final long TIMEOUT = 120L * 1000L;
+
+	private final KubeClientFactory kubeClientFactory = new DefaultKubeClientFactory();
+
+	private static final String LEADER_CONFIGMAP_NAME_PREFIX = "leader-test-cluster";
+
+	@Test
+	public void testMultipleKubernetesLeaderElectors() throws Exception {
+		final Configuration configuration = kubernetesResource.getConfiguration();
+		final ExecutorService executorService = kubernetesResource.getExecutorService();
+
+		final BlockingQueue<String> leaderStore = new LinkedBlockingQueue<>();
+		final String leaderConfigMapName = LEADER_CONFIGMAP_NAME_PREFIX + System.currentTimeMillis();
+		final int leaderNum = 3;
+
+		final KubernetesLeaderElector[] leaderElectors = new KubernetesLeaderElector[leaderNum];
+		// We use different Kubernetes clients for different leader electors.
+		final FlinkKubeClient[] kubeClients = new FlinkKubeClient[leaderNum];
+		final TestingLeaderCallbackHandler[] leaderCallbackHandlers = new TestingLeaderCallbackHandler[leaderNum];
+
+		try {
+			for (int i = 0; i < leaderNum; i++) {
+				kubeClients[i] = kubeClientFactory.fromConfiguration(configuration, executorService);
+				leaderCallbackHandlers[i] = new TestingLeaderCallbackHandler(leaderStore, UUID.randomUUID().toString());
+				final KubernetesLeaderElectionConfiguration leaderConfig = new KubernetesLeaderElectionConfiguration(
+					leaderConfigMapName, leaderCallbackHandlers[i].getLockIdentity(), configuration);
+				leaderElectors[i] = kubeClients[i].createLeaderElector(leaderConfig, leaderCallbackHandlers[i]);
+
+				// Start the leader electors to contend the leader
+				leaderElectors[i].run();
+			}
+
+			// Wait for the first leader
+			final String firstLockIdentity = leaderStore.poll(TIMEOUT, TimeUnit.MILLISECONDS);
+
+			for (int i = 0; i < leaderNum; i++) {
+				if (leaderCallbackHandlers[i].getLockIdentity().equals(firstLockIdentity)) {
+					// Check the callback is called.
+					assertThat(leaderCallbackHandlers[i].hasLeadership(), is(true));
+					// Current leader died
+					leaderElectors[i].stop();

Review comment:
       From the following piece of codes, we could find that `notLeader` will be called when the renew timeout or is interrupted. I will update the test to also verify the `notLeader` is called.
   
   ```
     /**
      * Starts the leader election loop
      */
     public void run() {
       LOGGER.debug("Leader election started");
       if (!acquire()) {
         return;
       }
       leaderElectionConfig.getLeaderCallbacks().onStartLeading();
       renewWithTimeout();
       leaderElectionConfig.getLeaderCallbacks().onStopLeading();
     }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507403885



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,58 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateAndDeleteConfigMap() {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(haLabels);
+		assertThat(this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildHAConfigMap());
+
+		final Supplier<Exception> configMapNotExistException = () -> new Exception("ConfigMap not exist");
+		FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function = c -> {
+			c.getData().put(LEADER_ADDRESS_KEY, LEADER_ADDRESS_NEW);
+			return c;
+		};
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+
+		// Checker not pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(LEADER_CONFIG_MAP_NAME, c -> false, function).get();
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(LEADER_CONFIG_MAP_NAME, c -> true, function).get();
+		this.flinkKubeClient.getConfigMap(LEADER_CONFIG_MAP_NAME).map(
+			configMap -> {
+				assertThat(configMap.getData().get(LEADER_ADDRESS_KEY), is(LEADER_ADDRESS_NEW));
+				return configMap;
+			}
+		).orElseThrow(configMapNotExistException);
+	}

Review comment:
       Actually, I change the implementation from `assertThat(configMapOpt.isPresent(), true);` to throw `configMapNotExistException`. Since it will make the test more concise. But now I could get them back and split into two cases.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r510988071



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
##########
@@ -137,8 +137,11 @@
 			operationFuture.whenCompleteAsync(
 				(t, throwable) -> {
 					if (throwable != null) {
+						final Throwable strippedThrowable = ExceptionUtils.stripCompletionException(throwable);
 						if (throwable instanceof CancellationException) {
 							resultFuture.completeExceptionally(new RetryException("Operation future was cancelled.", throwable));
+						} else if (strippedThrowable instanceof NotRetryException) {

Review comment:
       If we do this for the `retryOperation`, then we should also do it for the `retryOperationWithDelay` and all others which are retrying if an exception occurs.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private final LeaderElectionDriver leaderElectionDriver;
+
+	private final StateHandler leaderElectionStateHandle;
+
+	public DefaultLeaderElectionService(LeaderElectionDriver leaderElectionDriver) {
+		this.leaderElectionDriver = leaderElectionDriver;
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		leaderElectionStateHandle = new StateHandler();
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		LOG.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			leaderElectionDriver.start(leaderElectionStateHandle);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.stop();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+					leaderElectionDriver.writeLeaderInformation(confirmedLeaderSessionID, confirmedLeaderAddress);

Review comment:
       This call could be moved into `confirmLeaderInformation`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +231,73 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.exceptionally(throwable -> {
+				if (throwable != null) {
+					throw new CompletionException(
+						new KubernetesException("Failed to create ConfigMap " + configMapName, throwable));
+				}
+				return null;

Review comment:
       no need for the return statement if you get rid of the if statement.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java
##########
@@ -159,6 +159,38 @@ public void testRetryCancellation() throws Exception {
 		}
 	}
 
+	/**
+	 * Test that {@link FutureUtils#retry} should stop at {@link FutureUtils.NotRetryException}.
+	 */
+	@Test
+	public void testStopAtNotRetryException() {
+		final int retries = 10;
+		final int notRetry = 3;
+		final AtomicInteger atomicInteger = new AtomicInteger(0);
+		final String notRetryExceptionMsg = "Do not need to retry.";
+		CompletableFuture<Boolean> retryFuture = FutureUtils.retry(
+			() ->
+				CompletableFuture.supplyAsync(
+					() -> {
+						if (atomicInteger.incrementAndGet() == notRetry) {
+							throw new CompletionException(new FutureUtils.NotRetryException(notRetryExceptionMsg));

Review comment:
       maybe we could provide a utility `<T> T FutureUtils.stopRetry(String message)` which exactly does what you are doing here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private final LeaderElectionDriver leaderElectionDriver;
+
+	private final StateHandler leaderElectionStateHandle;
+
+	public DefaultLeaderElectionService(LeaderElectionDriver leaderElectionDriver) {
+		this.leaderElectionDriver = leaderElectionDriver;
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		leaderElectionStateHandle = new StateHandler();
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		LOG.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			leaderElectionDriver.start(leaderElectionStateHandle);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.stop();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+					leaderElectionDriver.writeLeaderInformation(confirmedLeaderSessionID, confirmedLeaderAddress);
+				} else {
+					LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"LeaderElectionService has already been stopped.", leaderSessionID);
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+						"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	/**
+	 * Helper class for the specific {@link LeaderElectionDriver} to operate the internal state.
+	 */
+	public class StateHandler {
+
+		/**
+		 * Called by specific {@link LeaderElectionDriver} when the leadership is granted.
+		 */
+		@GuardedBy("lock")
+		public void onGrantLeadership() {
+			synchronized (lock) {
+				if (running) {
+					issuedLeaderSessionID = UUID.randomUUID();
+					clearConfirmedLeaderInformation();
+
+					if (LOG.isDebugEnabled()) {
+						LOG.debug(
+							"Grant leadership to contender {} with session ID {}.",
+							leaderContender.getDescription(),
+							issuedLeaderSessionID);
+					}
+
+					leaderContender.grantLeadership(issuedLeaderSessionID);
+				} else {
+					LOG.debug("Ignoring the grant leadership notification since the service has " +
+						"already been stopped.");
+				}
+			}
+		}
+
+		/**
+		 * Called by specific {@link LeaderElectionDriver} when the leadership is revoked.
+		 */
+		@GuardedBy("lock")
+		public void onRevokeLeadership() {
+			synchronized (lock) {
+				if (running) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+
+					issuedLeaderSessionID = null;
+					clearConfirmedLeaderInformation();
+
+					leaderContender.revokeLeadership();
+				} else {
+					LOG.debug("Ignoring the revoke leadership notification since the service " +
+						"has already been stopped.");
+				}
+			}
+		}
+
+		/**
+		 * Called by specific {@link LeaderElectionDriver} when it wants to use leader information to do some
+		 * operations. For example, correct the external storage when the leader information is updated exceptionally.
+		 *
+		 * @param consumer to specify the operation. The exception will be handled by leader contender.
+		 */
+		@GuardedBy("lock")
+		public void runWithLock(BiConsumerWithException<UUID, String, Exception> consumer) {
+			try {
+				synchronized (lock) {
+					consumer.accept(confirmedLeaderSessionID, confirmedLeaderAddress);
+				}
+			} catch (Exception e) {
+				handleError(e);
+			}
+		}

Review comment:
       Instead of providing a general purpose method to run code, maybe we could add a callback `onLeaderInformationChange(LeaderInformation)` which is called by the driver. Then the election service can take the appropriate action (e.g. write the leader information again if necessary).

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private final LeaderElectionDriver leaderElectionDriver;
+
+	private final StateHandler leaderElectionStateHandle;
+
+	public DefaultLeaderElectionService(LeaderElectionDriver leaderElectionDriver) {
+		this.leaderElectionDriver = leaderElectionDriver;
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		leaderElectionStateHandle = new StateHandler();
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		LOG.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			leaderElectionDriver.start(leaderElectionStateHandle);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.stop();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+					leaderElectionDriver.writeLeaderInformation(confirmedLeaderSessionID, confirmedLeaderAddress);
+				} else {
+					LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"LeaderElectionService has already been stopped.", leaderSessionID);
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+						"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	/**
+	 * Helper class for the specific {@link LeaderElectionDriver} to operate the internal state.
+	 */
+	public class StateHandler {

Review comment:
       Maybe we could call it `LeaderEventHandler` or so.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private final LeaderElectionDriver leaderElectionDriver;
+
+	private final StateHandler leaderElectionStateHandle;
+
+	public DefaultLeaderElectionService(LeaderElectionDriver leaderElectionDriver) {
+		this.leaderElectionDriver = leaderElectionDriver;
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		leaderElectionStateHandle = new StateHandler();
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		LOG.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			leaderElectionDriver.start(leaderElectionStateHandle);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.stop();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+					leaderElectionDriver.writeLeaderInformation(confirmedLeaderSessionID, confirmedLeaderAddress);
+				} else {
+					LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"LeaderElectionService has already been stopped.", leaderSessionID);
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+						"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	public UUID getLeaderSessionID() {

Review comment:
       `@Nullable` is missing

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private final LeaderElectionDriver leaderElectionDriver;
+
+	private final StateHandler leaderElectionStateHandle;
+
+	public DefaultLeaderElectionService(LeaderElectionDriver leaderElectionDriver) {
+		this.leaderElectionDriver = leaderElectionDriver;
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		leaderElectionStateHandle = new StateHandler();
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		LOG.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			leaderElectionDriver.start(leaderElectionStateHandle);

Review comment:
       We might make the `LeaderElectionDriver` a bit easier if we instantiated it here. Then we could save the `start` method. Of course we would need to add a `LeaderElectionDriverFactory`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private final LeaderElectionDriver leaderElectionDriver;
+
+	private final StateHandler leaderElectionStateHandle;
+
+	public DefaultLeaderElectionService(LeaderElectionDriver leaderElectionDriver) {
+		this.leaderElectionDriver = leaderElectionDriver;
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		leaderElectionStateHandle = new StateHandler();
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		LOG.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			leaderElectionDriver.start(leaderElectionStateHandle);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.stop();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+					leaderElectionDriver.writeLeaderInformation(confirmedLeaderSessionID, confirmedLeaderAddress);
+				} else {
+					LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"LeaderElectionService has already been stopped.", leaderSessionID);
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+						"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	/**
+	 * Helper class for the specific {@link LeaderElectionDriver} to operate the internal state.
+	 */
+	public class StateHandler {
+
+		/**
+		 * Called by specific {@link LeaderElectionDriver} when the leadership is granted.
+		 */
+		@GuardedBy("lock")
+		public void onGrantLeadership() {
+			synchronized (lock) {
+				if (running) {
+					issuedLeaderSessionID = UUID.randomUUID();
+					clearConfirmedLeaderInformation();
+
+					if (LOG.isDebugEnabled()) {
+						LOG.debug(
+							"Grant leadership to contender {} with session ID {}.",
+							leaderContender.getDescription(),
+							issuedLeaderSessionID);
+					}
+
+					leaderContender.grantLeadership(issuedLeaderSessionID);
+				} else {
+					LOG.debug("Ignoring the grant leadership notification since the service has " +
+						"already been stopped.");
+				}
+			}
+		}
+
+		/**
+		 * Called by specific {@link LeaderElectionDriver} when the leadership is revoked.
+		 */
+		@GuardedBy("lock")
+		public void onRevokeLeadership() {
+			synchronized (lock) {
+				if (running) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+
+					issuedLeaderSessionID = null;
+					clearConfirmedLeaderInformation();
+
+					leaderContender.revokeLeadership();
+				} else {
+					LOG.debug("Ignoring the revoke leadership notification since the service " +
+						"has already been stopped.");
+				}
+			}
+		}
+
+		/**
+		 * Called by specific {@link LeaderElectionDriver} when it wants to use leader information to do some
+		 * operations. For example, correct the external storage when the leader information is updated exceptionally.
+		 *
+		 * @param consumer to specify the operation. The exception will be handled by leader contender.
+		 */
+		@GuardedBy("lock")
+		public void runWithLock(BiConsumerWithException<UUID, String, Exception> consumer) {
+			try {
+				synchronized (lock) {
+					consumer.accept(confirmedLeaderSessionID, confirmedLeaderAddress);
+				}
+			} catch (Exception e) {
+				handleError(e);
+			}
+		}
+
+		/**
+		 * Handle error by specific {@link LeaderElectionDriver}.
+		 * @param ex exception to be handled.
+		 */
+		public void handleError(Exception ex) {
+			leaderContender.handleError(ex);
+		}
+
+		/**
+		 * @return the contender description.
+		 */
+		public String getLeaderContenderDescription() {
+			return leaderContender.getDescription();
+		}

Review comment:
       Do we really need this here?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -38,24 +37,22 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.annotation.Nonnull;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.util.UUID;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
- * Leader election service for multiple JobManager. The leading JobManager is elected using
+ * {@link LeaderElectionDriver} implemented by Zookeeper. The leading JobManager is elected using

Review comment:
       `{@link LeaderElectionDriver} implementation for Zookeeper.`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private final LeaderElectionDriver leaderElectionDriver;
+
+	private final StateHandler leaderElectionStateHandle;
+
+	public DefaultLeaderElectionService(LeaderElectionDriver leaderElectionDriver) {
+		this.leaderElectionDriver = leaderElectionDriver;
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		leaderElectionStateHandle = new StateHandler();
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		LOG.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			leaderElectionDriver.start(leaderElectionStateHandle);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.stop();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+					leaderElectionDriver.writeLeaderInformation(confirmedLeaderSessionID, confirmedLeaderAddress);
+				} else {
+					LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"LeaderElectionService has already been stopped.", leaderSessionID);
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+						"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	/**
+	 * Helper class for the specific {@link LeaderElectionDriver} to operate the internal state.
+	 */
+	public class StateHandler {

Review comment:
       For easier testability of the driver implementations, I would suggest to make this an interface so that one can provide a testing implementation.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalDriver.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.leaderretrieval;
+
+/**
+ * A {@link LeaderRetrievalDriver} is responsible for retrieves the current leader which has been elected by the
+ * {@link LeaderRetrievalDriver}.
+ */
+public interface LeaderRetrievalDriver {
+
+	/**
+	 * Start the necessary services for specific {@link LeaderRetrievalDriver} implementations. For example, NodeCache
+	 * in Zookeeper, ConfigMap watcher in Kubernetes. They could get the leader information update events and need to
+	 * notify the leader listener by {@link DefaultLeaderRetrievalService.StateHandler}.
+	 *
+	 * @param leaderRetrievalStateHandler used for notify the leader changes.
+	 * @throws Exception Throw exception when start the services.
+	 */
+	void start(DefaultLeaderRetrievalService.StateHandler leaderRetrievalStateHandler) throws Exception;
+
+	/**
+	 * Stop the services used for leader retrieval.
+	 */
+	void stop() throws Exception;

Review comment:
       I'd say to call this method `close` instead of `stop`. We could let this interface also extend the `AutoCloseable` interface.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve the leader information from
+ * different storage. The leader address as well as the current leader session ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService {
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private String lastLeaderAddress;
+
+	private UUID lastLeaderSessionID;
+
+	private volatile boolean running;
+
+	/** Listener which will be notified about leader changes. */
+	private volatile LeaderRetrievalListener leaderListener;
+
+	private final LeaderRetrievalDriver leaderRetrievalDriver;
+
+	private final StateHandler leaderRetrievalStateHandler;

Review comment:
       I'd suggest to keep final fields grouped together. That way it is easier to see what is mutable state of a class.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve the leader information from
+ * different storage. The leader address as well as the current leader session ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService {
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private String lastLeaderAddress;
+
+	private UUID lastLeaderSessionID;
+
+	private volatile boolean running;
+
+	/** Listener which will be notified about leader changes. */
+	private volatile LeaderRetrievalListener leaderListener;
+
+	private final LeaderRetrievalDriver leaderRetrievalDriver;
+
+	private final StateHandler leaderRetrievalStateHandler;
+
+	/**
+	 * Creates a default leader retrieval service with specified {@link LeaderRetrievalDriver}.
+	 *
+	 * @param leaderRetrievalDriver {@link LeaderRetrievalDriver} implemented by Zookeeper, Kubernetes, etc.
+	 */
+	public DefaultLeaderRetrievalService(LeaderRetrievalDriver leaderRetrievalDriver) {
+		this.leaderRetrievalDriver = leaderRetrievalDriver;
+
+		this.lastLeaderAddress = null;
+		this.lastLeaderSessionID = null;
+
+		running = false;
+
+		this.leaderRetrievalStateHandler = new StateHandler();
+	}
+
+	@Override
+	public void start(LeaderRetrievalListener listener) throws Exception {
+		Preconditions.checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "DefaultLeaderRetrievalService can " +
+			"only be started once.");
+
+		LOG.info("Starting DefaultLeaderRetrievalService with {}.", leaderRetrievalDriver);
+
+		synchronized (lock) {
+			leaderListener = listener;
+			leaderRetrievalDriver.start(leaderRetrievalStateHandler);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Stopping DefaultLeaderRetrievalService {} with {}.", leaderRetrievalDriver);
+
+			leaderRetrievalDriver.stop();
+		}
+	}
+
+	/**
+	 * Helper class for the specific {@link LeaderRetrievalDriver} to notify the leader changes.
+	 */
+	public class StateHandler {
+
+		/**
+		 * Called by specific {@link LeaderRetrievalDriver} to notify new leader address.
+		 * @param supplier supplier to provide a {@link Tuple2} of leader information. f0 is UUID, and f1 is leader
+		 * address. The exception will be handled by leader listener.
+		 */
+		@GuardedBy("lock")
+		public void notifyIfNewLeaderAddress(SupplierWithException<Tuple2<UUID, String>, Exception> supplier) {

Review comment:
       Let's call this method `notifyLeaderAddress` and let the `DefaultLeaderRetrievalService` be responsible for deciding whether to call the listener or not.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve the leader information from
+ * different storage. The leader address as well as the current leader session ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService {
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private String lastLeaderAddress;
+
+	private UUID lastLeaderSessionID;
+
+	private volatile boolean running;
+
+	/** Listener which will be notified about leader changes. */
+	private volatile LeaderRetrievalListener leaderListener;
+
+	private final LeaderRetrievalDriver leaderRetrievalDriver;
+
+	private final StateHandler leaderRetrievalStateHandler;
+
+	/**
+	 * Creates a default leader retrieval service with specified {@link LeaderRetrievalDriver}.
+	 *
+	 * @param leaderRetrievalDriver {@link LeaderRetrievalDriver} implemented by Zookeeper, Kubernetes, etc.
+	 */
+	public DefaultLeaderRetrievalService(LeaderRetrievalDriver leaderRetrievalDriver) {
+		this.leaderRetrievalDriver = leaderRetrievalDriver;
+
+		this.lastLeaderAddress = null;
+		this.lastLeaderSessionID = null;
+
+		running = false;
+
+		this.leaderRetrievalStateHandler = new StateHandler();
+	}
+
+	@Override
+	public void start(LeaderRetrievalListener listener) throws Exception {
+		Preconditions.checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "DefaultLeaderRetrievalService can " +
+			"only be started once.");
+
+		LOG.info("Starting DefaultLeaderRetrievalService with {}.", leaderRetrievalDriver);
+
+		synchronized (lock) {
+			leaderListener = listener;
+			leaderRetrievalDriver.start(leaderRetrievalStateHandler);

Review comment:
       Same here, maybe we can pass a `LeaderRetrievalDriverFactory` to this class and then pass in the `leaderRetrievalStateHandler` when we create the actual `leaderRetrievalDriver`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private final LeaderElectionDriver leaderElectionDriver;
+
+	private final StateHandler leaderElectionStateHandle;
+
+	public DefaultLeaderElectionService(LeaderElectionDriver leaderElectionDriver) {
+		this.leaderElectionDriver = leaderElectionDriver;
+
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		leaderElectionStateHandle = new StateHandler();
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		LOG.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			leaderElectionDriver.start(leaderElectionStateHandle);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		LOG.info("Stopping LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			leaderElectionDriver.stop();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		synchronized (lock) {
+			if (hasLeadership(leaderSessionID)) {
+				if (running) {
+					confirmLeaderInformation(leaderSessionID, leaderAddress);
+					leaderElectionDriver.writeLeaderInformation(confirmedLeaderSessionID, confirmedLeaderAddress);
+				} else {
+					LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"LeaderElectionService has already been stopped.", leaderSessionID);
+				}
+			} else {
+				// Received an old confirmation call
+				if (!leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					LOG.debug("Receive an old confirmation call of leader session ID {}, " +
+						"current issued session ID is {}", leaderSessionID, issuedLeaderSessionID);
+				} else {
+					LOG.warn("The leader session ID {} was confirmed even though the " +
+						"corresponding JobManager was not elected as the leader.", leaderSessionID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		synchronized (lock) {
+			return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+		}
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	@VisibleForTesting
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@GuardedBy("lock")
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+	}
+
+	@GuardedBy("lock")
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}
+
+	/**
+	 * Helper class for the specific {@link LeaderElectionDriver} to operate the internal state.
+	 */
+	public class StateHandler {
+
+		/**
+		 * Called by specific {@link LeaderElectionDriver} when the leadership is granted.
+		 */
+		@GuardedBy("lock")
+		public void onGrantLeadership() {
+			synchronized (lock) {
+				if (running) {
+					issuedLeaderSessionID = UUID.randomUUID();
+					clearConfirmedLeaderInformation();
+
+					if (LOG.isDebugEnabled()) {
+						LOG.debug(
+							"Grant leadership to contender {} with session ID {}.",
+							leaderContender.getDescription(),
+							issuedLeaderSessionID);
+					}
+
+					leaderContender.grantLeadership(issuedLeaderSessionID);
+				} else {
+					LOG.debug("Ignoring the grant leadership notification since the service has " +
+						"already been stopped.");
+				}
+			}
+		}
+
+		/**
+		 * Called by specific {@link LeaderElectionDriver} when the leadership is revoked.
+		 */
+		@GuardedBy("lock")
+		public void onRevokeLeadership() {
+			synchronized (lock) {
+				if (running) {
+					LOG.debug(
+						"Revoke leadership of {} ({}@{}).",
+						leaderContender.getDescription(),
+						confirmedLeaderSessionID,
+						confirmedLeaderAddress);
+
+					issuedLeaderSessionID = null;
+					clearConfirmedLeaderInformation();
+
+					leaderContender.revokeLeadership();
+				} else {
+					LOG.debug("Ignoring the revoke leadership notification since the service " +
+						"has already been stopped.");
+				}
+			}
+		}
+
+		/**
+		 * Called by specific {@link LeaderElectionDriver} when it wants to use leader information to do some
+		 * operations. For example, correct the external storage when the leader information is updated exceptionally.
+		 *
+		 * @param consumer to specify the operation. The exception will be handled by leader contender.
+		 */
+		@GuardedBy("lock")
+		public void runWithLock(BiConsumerWithException<UUID, String, Exception> consumer) {
+			try {
+				synchronized (lock) {
+					consumer.accept(confirmedLeaderSessionID, confirmedLeaderAddress);
+				}
+			} catch (Exception e) {
+				handleError(e);
+			}
+		}
+
+		/**
+		 * Handle error by specific {@link LeaderElectionDriver}.
+		 * @param ex exception to be handled.
+		 */
+		public void handleError(Exception ex) {
+			leaderContender.handleError(ex);
+		}
+
+		/**
+		 * @return the contender description.
+		 */
+		public String getLeaderContenderDescription() {
+			return leaderContender.getDescription();
+		}

Review comment:
       Instead of exposing this via the `StateHandler`, we could pass this also to the Driver's constructor in the `DefaultLeaderElectionService.start(LeaderContender)` method.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -169,60 +171,61 @@ public static String getZooKeeperEnsemble(Configuration flinkConf)
 	}
 
 	/**
-	 * Creates a {@link ZooKeeperLeaderRetrievalService} instance.
+	 * Creates a {@link DefaultLeaderRetrievalService} instance with{@link ZookeeperLeaderRetrievalDriver}.

Review comment:
       Whitespace is missing after `with`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZookeeperLeaderRetrievalDriver.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link ZooKeeperLeaderElectionDriver}.
+ * {@link LeaderRetrievalService} implemented by Zookeeper. It retrieves the current leader which has

Review comment:
       `{@link LeaderRetrievalService} implementation for Zookeeper.`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve the leader information from
+ * different storage. The leader address as well as the current leader session ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService {
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private String lastLeaderAddress;
+
+	private UUID lastLeaderSessionID;
+
+	private volatile boolean running;
+
+	/** Listener which will be notified about leader changes. */
+	private volatile LeaderRetrievalListener leaderListener;
+
+	private final LeaderRetrievalDriver leaderRetrievalDriver;
+
+	private final StateHandler leaderRetrievalStateHandler;
+
+	/**
+	 * Creates a default leader retrieval service with specified {@link LeaderRetrievalDriver}.
+	 *
+	 * @param leaderRetrievalDriver {@link LeaderRetrievalDriver} implemented by Zookeeper, Kubernetes, etc.
+	 */
+	public DefaultLeaderRetrievalService(LeaderRetrievalDriver leaderRetrievalDriver) {
+		this.leaderRetrievalDriver = leaderRetrievalDriver;
+
+		this.lastLeaderAddress = null;
+		this.lastLeaderSessionID = null;
+
+		running = false;
+
+		this.leaderRetrievalStateHandler = new StateHandler();
+	}
+
+	@Override
+	public void start(LeaderRetrievalListener listener) throws Exception {
+		Preconditions.checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "DefaultLeaderRetrievalService can " +
+			"only be started once.");
+
+		LOG.info("Starting DefaultLeaderRetrievalService with {}.", leaderRetrievalDriver);
+
+		synchronized (lock) {
+			leaderListener = listener;
+			leaderRetrievalDriver.start(leaderRetrievalStateHandler);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Stopping DefaultLeaderRetrievalService {} with {}.", leaderRetrievalDriver);
+
+			leaderRetrievalDriver.stop();
+		}
+	}
+
+	/**
+	 * Helper class for the specific {@link LeaderRetrievalDriver} to notify the leader changes.
+	 */
+	public class StateHandler {
+
+		/**
+		 * Called by specific {@link LeaderRetrievalDriver} to notify new leader address.
+		 * @param supplier supplier to provide a {@link Tuple2} of leader information. f0 is UUID, and f1 is leader
+		 * address. The exception will be handled by leader listener.
+		 */
+		@GuardedBy("lock")
+		public void notifyIfNewLeaderAddress(SupplierWithException<Tuple2<UUID, String>, Exception> supplier) {
+			try {
+				final Tuple2<UUID, String> leaderInformation = supplier.get();
+				final UUID newLeaderSessionID = leaderInformation.f0;
+				final String newLeaderAddress = leaderInformation.f1;
+				if (!(Objects.equals(newLeaderAddress, lastLeaderAddress) &&
+					Objects.equals(newLeaderSessionID, lastLeaderSessionID))) {
+					if (newLeaderAddress == null && newLeaderSessionID == null) {
+						LOG.debug("Leader information was lost: The listener will be notified accordingly.");
+					} else {
+						LOG.debug(
+							"New leader information: Leader={}, session ID={}.",
+							newLeaderAddress,
+							newLeaderSessionID);
+					}
+
+					lastLeaderAddress = newLeaderAddress;
+					lastLeaderSessionID = newLeaderSessionID;
+					leaderListener.notifyLeaderAddress(newLeaderAddress, newLeaderSessionID);
+				}
+			} catch (Exception e) {
+				leaderListener.handleError(e);
+			}
+		}
+
+		/**
+		 * Called by specific {@link LeaderRetrievalDriver} to notify leader loss.
+		 */
+		@GuardedBy("lock")
+		public void notifyLeaderLoss() {
+			notifyIfNewLeaderAddress(() -> new Tuple2<>(null, null));
+		}

Review comment:
       Isn't this call not redundant? Couldn't we use the `notifyLeaderAddress(LeaderAddressInformation)` which we call with `notifyLeaderAddress(LeaderAddressInformation.empty())` or so?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionDriver.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.leaderelection;
+
+import java.util.UUID;
+
+/**
+ * A {@link LeaderElectionDriver} is responsible for performing the leader election and storing the leader information.
+ * All the leader internal state is guarded by lock in {@link DefaultLeaderElectionService}. Different driver
+ * implementations do not need to care about the lock. And it should use {@link DefaultLeaderElectionService.StateHandler}
+ * if it want to access the internal leader state.
+ */
+public interface LeaderElectionDriver {
+
+	/**
+	 * Start the necessary services for specific {@link LeaderElectionService} implementations. For example, LeaderLatch
+	 * and NodeCache in Zookeeper, KubernetesLeaderElector and ConfigMap watcher in Kubernetes. When the leader election
+	 * finished, the leaderElectionStateHandler will be used to grant/revoke leadership.
+	 *
+	 * @param leaderElectionStateHandler used for operating the internal leader state.
+	 * @throws Exception Throw exception when start the services.
+	 */
+	void start(DefaultLeaderElectionService.StateHandler leaderElectionStateHandler) throws Exception;
+
+	/**
+	 * Stop the services used for leader election.
+	 */
+	void stop() throws Exception;

Review comment:
       maybe let's call it `close`. Otherwise one might think that one can start-stop-start the driver.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZookeeperLeaderRetrievalDriver.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link ZooKeeperLeaderElectionDriver}.
+ * {@link LeaderRetrievalService} implemented by Zookeeper. It retrieves the current leader which has
+ * been elected by the {@link ZooKeeperLeaderElectionDriver}.
+ * The leader address as well as the current leader session ID is retrieved from ZooKeeper.
+ */
+public class ZookeeperLeaderRetrievalDriver implements LeaderRetrievalDriver, NodeCacheListener, UnhandledErrorListener {
+	private static final Logger LOG = LoggerFactory.getLogger(ZookeeperLeaderRetrievalDriver.class);
+
+	/** Connection to the used ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe to watch changes of a specific ZooKeeper node. */
+	private final NodeCache cache;
+
+	private final String retrievalPath;
+
+	private volatile boolean running;
+
+	private final ConnectionStateListener connectionStateListener = (client, newState) -> handleStateChange(newState);
+
+	private DefaultLeaderRetrievalService.StateHandler leaderRetrievalStateHandler;
+
+	/**
+	 * Creates a leader retrieval service which uses ZooKeeper to retrieve the leader information.
+	 *
+	 * @param client Client which constitutes the connection to the ZooKeeper quorum
+	 * @param retrievalPath Path of the ZooKeeper node which contains the leader information
+	 */
+	public ZookeeperLeaderRetrievalDriver(CuratorFramework client, String retrievalPath) {
+		this.client = checkNotNull(client, "CuratorFramework client");
+		this.cache = new NodeCache(client, retrievalPath);
+		this.retrievalPath = checkNotNull(retrievalPath);
+
+		running = false;
+	}
+
+	@Override
+	public void start(DefaultLeaderRetrievalService.StateHandler stateHandler) throws Exception {
+		LOG.info("Starting {}.", this);
+
+		this.leaderRetrievalStateHandler = checkNotNull(stateHandler);
+
+		client.getUnhandledErrorListenable().addListener(this);
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(connectionStateListener);
+
+		running = true;
+	}
+
+	@Override
+	public void stop() throws Exception {
+		if (!running) {
+			return;
+		}
+
+		running = false;
+
+		LOG.info("Stopping {}.", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+		client.getConnectionStateListenable().removeListener(connectionStateListener);
+
+		try {
+			cache.close();
+		} catch (IOException e) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderRetrievalService.", e);
+		}
+	}
+
+	@Override
+	public void nodeChanged() {
+		if (running) {
+			leaderRetrievalStateHandler.notifyIfNewLeaderAddress(() -> {
+				LOG.debug("Leader node has changed.");
+
+				final ChildData childData = cache.getCurrentData();
+
+				final String leaderAddress;
+				final UUID leaderSessionID;
+
+				if (childData == null) {
+					leaderAddress = null;
+					leaderSessionID = null;
+				} else {
+					byte[] data = childData.getData();
+
+					if (data == null || data.length == 0) {
+						leaderAddress = null;
+						leaderSessionID = null;
+					} else {
+						ByteArrayInputStream bais = new ByteArrayInputStream(data);
+						ObjectInputStream ois = new ObjectInputStream(bais);
+
+						leaderAddress = ois.readUTF();
+						leaderSessionID = (UUID) ois.readObject();
+					}
+				}
+				return new Tuple2<>(leaderSessionID, leaderAddress);

Review comment:
       I think the whole address extraction does not have to happen under the lock. I'd suggest to do it first and then simply call `LeaderRetrievalEventHandler.notifyLeaderAddress`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve the leader information from
+ * different storage. The leader address as well as the current leader session ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService {
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private String lastLeaderAddress;
+
+	private UUID lastLeaderSessionID;
+
+	private volatile boolean running;
+
+	/** Listener which will be notified about leader changes. */
+	private volatile LeaderRetrievalListener leaderListener;
+
+	private final LeaderRetrievalDriver leaderRetrievalDriver;
+
+	private final StateHandler leaderRetrievalStateHandler;
+
+	/**
+	 * Creates a default leader retrieval service with specified {@link LeaderRetrievalDriver}.
+	 *
+	 * @param leaderRetrievalDriver {@link LeaderRetrievalDriver} implemented by Zookeeper, Kubernetes, etc.
+	 */
+	public DefaultLeaderRetrievalService(LeaderRetrievalDriver leaderRetrievalDriver) {
+		this.leaderRetrievalDriver = leaderRetrievalDriver;
+
+		this.lastLeaderAddress = null;
+		this.lastLeaderSessionID = null;
+
+		running = false;
+
+		this.leaderRetrievalStateHandler = new StateHandler();
+	}
+
+	@Override
+	public void start(LeaderRetrievalListener listener) throws Exception {
+		Preconditions.checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "DefaultLeaderRetrievalService can " +
+			"only be started once.");
+
+		LOG.info("Starting DefaultLeaderRetrievalService with {}.", leaderRetrievalDriver);
+
+		synchronized (lock) {
+			leaderListener = listener;
+			leaderRetrievalDriver.start(leaderRetrievalStateHandler);
+
+			running = true;
+		}
+	}
+
+	@Override
+	public void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Stopping DefaultLeaderRetrievalService {} with {}.", leaderRetrievalDriver);
+
+			leaderRetrievalDriver.stop();
+		}
+	}
+
+	/**
+	 * Helper class for the specific {@link LeaderRetrievalDriver} to notify the leader changes.
+	 */
+	public class StateHandler {

Review comment:
       I would suggest to call it `LeaderRetrievalEventHandler` and make it an interface for easier testability.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
##########
@@ -70,7 +68,8 @@
 import static org.mockito.Mockito.when;
 
 /**
- * Tests for the {@link ZooKeeperLeaderElectionService} and the {@link ZooKeeperLeaderRetrievalService}.
+ * Tests for the {@link DefaultLeaderElectionService} with {@link ZooKeeperLeaderElectionDriver}
+ * and the {@link DefaultLeaderRetrievalService} with {@link org.apache.flink.runtime.leaderretrieval.ZookeeperLeaderRetrievalDriver}.
  */
 public class ZooKeeperLeaderElectionTest extends TestLogger {

Review comment:
       It might be possible to update some of the tests to work directly against the `ZooKeeperDrivers` instead of having to instantiate a the services.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -176,168 +135,92 @@ public void stop() throws Exception{
 		}
 
 		if (exception != null) {
-			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionService.", exception);
-		}
-	}
-
-	@Override
-	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(
-				"Confirm leader session ID {} for leader {}.",
-				leaderSessionID,
-				leaderAddress);
-		}
-
-		Preconditions.checkNotNull(leaderSessionID);
-
-		if (leaderLatch.hasLeadership()) {
-			// check if this is an old confirmation call
-			synchronized (lock) {
-				if (running) {
-					if (leaderSessionID.equals(this.issuedLeaderSessionID)) {
-						confirmLeaderInformation(leaderSessionID, leaderAddress);
-						writeLeaderInformation();
-					}
-				} else {
-					LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
-						"ZooKeeperLeaderElectionService has already been stopped.", leaderSessionID);
-				}
-			}
-		} else {
-			LOG.warn("The leader session ID {} was confirmed even though the " +
-					"corresponding JobManager was not elected as the leader.", leaderSessionID);
+			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
 		}
 	}
 
-	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
-		confirmedLeaderSessionID = leaderSessionID;
-		confirmedLeaderAddress = leaderAddress;
-	}
-
 	@Override
-	public boolean hasLeadership(@Nonnull UUID leaderSessionId) {
-		return leaderLatch.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+	public boolean hasLeadership() {
+		return leaderLatch.hasLeadership();
 	}
 
 	@Override
 	public void isLeader() {
-		synchronized (lock) {
-			if (running) {
-				issuedLeaderSessionID = UUID.randomUUID();
-				clearConfirmedLeaderInformation();
-
-				if (LOG.isDebugEnabled()) {
-					LOG.debug(
-						"Grant leadership to contender {} with session ID {}.",
-						leaderContender.getDescription(),
-						issuedLeaderSessionID);
-				}
-
-				leaderContender.grantLeadership(issuedLeaderSessionID);
-			} else {
-				LOG.debug("Ignoring the grant leadership notification since the service has " +
-					"already been stopped.");
-			}
-		}
-	}
-
-	private void clearConfirmedLeaderInformation() {
-		confirmedLeaderSessionID = null;
-		confirmedLeaderAddress = null;
+		leaderElectionStateHandler.onGrantLeadership();
 	}
 
 	@Override
 	public void notLeader() {
-		synchronized (lock) {
-			if (running) {
-				LOG.debug(
-					"Revoke leadership of {} ({}@{}).",
-					leaderContender.getDescription(),
-					confirmedLeaderSessionID,
-					confirmedLeaderAddress);
-
-				issuedLeaderSessionID = null;
-				clearConfirmedLeaderInformation();
-
-				leaderContender.revokeLeadership();
-			} else {
-				LOG.debug("Ignoring the revoke leadership notification since the service " +
-					"has already been stopped.");
-			}
-		}
+		leaderElectionStateHandler.onRevokeLeadership();
 	}
 
 	@Override
-	public void nodeChanged() throws Exception {
-		try {
+	public void nodeChanged() {
+		final String leaderContenderDesc = leaderElectionStateHandler.getLeaderContenderDescription();
+		leaderElectionStateHandler.runWithLock((confirmedLeaderSessionID, confirmedLeaderAddress) -> {
 			// leaderSessionID is null if the leader contender has not yet confirmed the session ID
 			if (leaderLatch.hasLeadership()) {
-				synchronized (lock) {
-					if (running) {
-						if (LOG.isDebugEnabled()) {
-							LOG.debug(
-								"Leader node changed while {} is the leader with session ID {}.",
-								leaderContender.getDescription(),
-								confirmedLeaderSessionID);
-						}
+				if (running) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug(
+							"Leader node changed while {} is the leader with session ID {}.",
+							leaderContenderDesc,
+							confirmedLeaderSessionID);
+					}
+
+					if (confirmedLeaderSessionID != null) {
+						ChildData childData = cache.getCurrentData();
 
-						if (confirmedLeaderSessionID != null) {
-							ChildData childData = cache.getCurrentData();
+						if (childData == null) {
+							if (LOG.isDebugEnabled()) {
+								LOG.debug(
+									"Writing leader information into empty node by {}.",
+									leaderContenderDesc);
+							}
+							writeLeaderInformation(confirmedLeaderSessionID, confirmedLeaderAddress);
+						} else {
+							byte[] data = childData.getData();
 
-							if (childData == null) {
+							if (data == null || data.length == 0) {
+								// the data field seems to be empty, rewrite information
 								if (LOG.isDebugEnabled()) {
 									LOG.debug(
-										"Writing leader information into empty node by {}.",
-										leaderContender.getDescription());
+										"Writing leader information into node with empty data field by {}.",
+										leaderContenderDesc);
 								}
-								writeLeaderInformation();
+								writeLeaderInformation(confirmedLeaderSessionID, confirmedLeaderAddress);
 							} else {
-								byte[] data = childData.getData();
+								ByteArrayInputStream bais = new ByteArrayInputStream(data);
+								ObjectInputStream ois = new ObjectInputStream(bais);
 
-								if (data == null || data.length == 0) {
-									// the data field seems to be empty, rewrite information
+								String leaderAddress = ois.readUTF();
+								UUID leaderSessionID = (UUID) ois.readObject();
+
+								if (!leaderAddress.equals(confirmedLeaderAddress) ||
+									(leaderSessionID == null || !leaderSessionID.equals(confirmedLeaderSessionID))) {
+									// the data field does not correspond to the expected leader information
 									if (LOG.isDebugEnabled()) {
 										LOG.debug(
-											"Writing leader information into node with empty data field by {}.",
-											leaderContender.getDescription());
-									}
-									writeLeaderInformation();
-								} else {
-									ByteArrayInputStream bais = new ByteArrayInputStream(data);
-									ObjectInputStream ois = new ObjectInputStream(bais);
-
-									String leaderAddress = ois.readUTF();
-									UUID leaderSessionID = (UUID) ois.readObject();
-
-									if (!leaderAddress.equals(confirmedLeaderAddress) ||
-										(leaderSessionID == null || !leaderSessionID.equals(confirmedLeaderSessionID))) {
-										// the data field does not correspond to the expected leader information
-										if (LOG.isDebugEnabled()) {
-											LOG.debug(
-												"Correcting leader information by {}.",
-												leaderContender.getDescription());
-										}
-										writeLeaderInformation();
+											"Correcting leader information by {}.",
+											leaderContenderDesc);

Review comment:
       I think it would be good to let the driver extract the leader information and then let the service decide how to handle it (e.g. whether it needs to rewrite the leader information or not).

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +231,73 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.exceptionally(throwable -> {
+				if (throwable != null) {
+					throw new CompletionException(
+						new KubernetesException("Failed to create ConfigMap " + configMapName, throwable));
+				}
+				return null;
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {

Review comment:
       I am not yet fully convinced that `FunctionWithException` is the right choice. For the sake of completeness I repeat what I have written on an older discussion: I think we should only do `discardState` or something similar after we are sure that we have updated the `configMap` properly (e.g. removed the entry).

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +231,73 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.exceptionally(throwable -> {
+				if (throwable != null) {

Review comment:
       `throwable` will be non null so there is no check needed.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
##########
@@ -412,6 +415,26 @@ public RetryException(Throwable cause) {
 		}
 	}
 
+	/**
+	 * The {@link #retry(Supplier, int, Executor)} attempts will stop at this exception.
+	 */
+	public static class NotRetryException extends Exception {

Review comment:
       Maybe `StopRetryException` could be a slightly better name.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507561819



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader
+ */
+@PublicEvolving
+public class KubernetesHighAvailabilityOptions {
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<String> HA_KUBERNETES_LEADER_SUFFIX =
+			key("high-availability.kubernetes.leader.suffix")
+			.stringType()
+			.defaultValue("leader")
+			.withDescription("The ConfigMap suffix of the leader which contains the URL to the leader and the " +
+				"current leader session ID. Leader elector will use the same ConfigMap for contending the lock.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_LEASE_DURATION =
+			key("high-availability.kubernetes.client.lease-duration")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(30))
+			.withDescription("Define the lease duration for the Kubernetes leader election in ms. The leader will " +
+				"continuously renew its lease time to indicate its existence. And the followers will do a lease " +
+				"checking against the current time. \"renewTime + leaseDuration > now\" means the leader is alive.");
+
+	@Documentation.Section(Documentation.Sections.EXPERT_KUBERNETES_HIGH_AVAILABILITY)
+	public static final ConfigOption<Duration> KUBERNETES_RENEW_DEADLINE =
+			key("high-availability.kubernetes.client.renew-deadline")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(15))
+			.withDescription("Defines the deadline when the leader tries to renew the lease in ms. If it could not " +
+				"succeed in the given time, the renew operation will be aborted.");

Review comment:
       Make sense.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r511274908



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,69 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateConfigMap() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+		final Optional<KubernetesConfigMap> currentOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(currentOpt.isPresent(), is(true));
+		assertThat(currentOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	@Test
+	public void testDeleteConfigMapByLabels() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(TESTING_LABELS);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testDeleteConfigMapByName() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(
+			TESTING_CONFIG_MAP_NAME,
+			c -> {
+				c.getData().put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_NEW_VALUE);
+				return Optional.of(c);
+			}).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_NEW_VALUE));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMapWithEmptyResult() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker not pass and return empty result
+		this.flinkKubeClient.checkAndUpdateConfigMap(TESTING_CONFIG_MAP_NAME, c -> Optional.empty()).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	private KubernetesConfigMap buildTestingConfigMap() {
+		final Map<String, String> data = new HashMap<>();
+		data.put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_VALUE);
+		return new KubernetesConfigMap(new ConfigMapBuilder()
+			.withNewMetadata()
+			.withName(TESTING_CONFIG_MAP_NAME)
+			.withLabels(TESTING_LABELS)
+			.endMetadata()
+			.withData(data).build());
+	}

Review comment:
       `checkAndUpdateConfigMap` is really an important interface. All the correctness depends on the result of this interface. So I will add a ITCase for it to verify the K8s behavior.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508569365



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	/** The executor to run external IO operations on. */
+	protected final Executor executor;
+
+	/** The runtime configuration. */
+	protected final Configuration configuration;
+
+	/** Store for arbitrary blobs. */
+	private final BlobStoreService blobStoreService;
+
+	public AbstractHaServices(Executor executor, Configuration config, BlobStoreService blobStoreService) {

Review comment:
       nit: I try to always pass `Configuration` first because it represents the part which configures the component.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).

Review comment:
       Maybe add a bit better description what this class implements (e.g. if I want to implement `HighAvailabilityServices` what does this class already solve for me?).

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	protected volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	protected volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	protected volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			internalStop();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (logger.isDebugEnabled()) {
+			logger.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		if (hasLeadership()) {
+			// check if this is an old confirmation call
+			synchronized (lock) {
+				if (running) {
+					if (leaderSessionID.equals(this.issuedLeaderSessionID)) {
+						confirmLeaderInformation(leaderSessionID, leaderAddress);
+						writeLeaderInformation();
+					}
+				} else {
+					logger.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"LeaderElectionService has already been stopped.", leaderSessionID);
+				}
+			}
+		} else {
+			logger.warn("The leader session ID {} was confirmed even though the " +
+				"corresponding JobManager was not elected as the leader.", leaderSessionID);
+		}
+	}
+
+	@Override
+	public final boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		return hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	/**
+	 * Start the necessary services for specific {@link LeaderElectionService} implementations. For example, LeaderLatch
+	 * and NodeCache in Zookeeper, KubernetesLeaderElector and ConfigMap watcher in Kubernetes. When the leader election
+	 * finished, the contender will be used to grant/revoke leadership.
+	 *
+	 * @param contender LeaderContender to take part in the leader election process.
+	 * @throws Exception Throw exception when start the services.
+	 */
+	@GuardedBy("lock")
+	protected abstract void internalStart(LeaderContender contender) throws Exception;
+
+	/**
+	 * Stop the services used for leader election.
+	 */
+	@GuardedBy("lock")
+	protected abstract void internalStop() throws Exception;
+
+	/**
+	 * Write the current leader information to external persistent storage(e.g. Zookeeper, Kubernetes ConfigMap). This
+	 * is a blocking IO operation.
+	 */
+	protected abstract void writeLeaderInformation();
+
+	/**
+	 * Check whether the current internal leader elector(e.g. Zookeeper LeaderLatch, KubernetesLeaderElector) still have
+	 * the leadership.
+	 *
+	 * @return Whether the current internal leader elector has the leadership.
+	 */
+	protected abstract boolean hasLeadership();
+
+	/**
+	 * Called by subclass when the leadership is granted.
+	 */
+	@GuardedBy("lock")
+	protected final void onGrantLeadership() {
+		synchronized (lock) {
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				clearConfirmedLeaderInformation();
+
+				if (logger.isDebugEnabled()) {
+					logger.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getDescription(),
+						issuedLeaderSessionID);
+				}
+
+				leaderContender.grantLeadership(issuedLeaderSessionID);
+			} else {
+				logger.debug("Ignoring the grant leadership notification since the service has " +
+					"already been stopped.");
+			}
+		}
+	}
+
+	/**
+	 * Called by subclass when the leadership is revoked.
+	 */
+	@GuardedBy("lock")
+	protected final void onRevokeLeadership() {
+		synchronized (lock) {
+			if (running) {
+				logger.debug(
+					"Revoke leadership of {} ({}@{}).",
+					leaderContender.getDescription(),
+					confirmedLeaderSessionID,
+					confirmedLeaderAddress);
+
+				issuedLeaderSessionID = null;
+				clearConfirmedLeaderInformation();
+
+				leaderContender.revokeLeadership();
+			} else {
+				logger.debug("Ignoring the revoke leadership notification since the service " +
+					"has already been stopped.");
+			}
+		}
+	}
+
+	private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) {
+		confirmedLeaderSessionID = leaderSessionID;
+		confirmedLeaderAddress = leaderAddress;
+	}
+
+	private void clearConfirmedLeaderInformation() {
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+	}

Review comment:
       I'd suggest to annotate these methods with `@GuardedBy("lock")`. Otherwise the analyzer will complain that the fields are accessed w/o having the lock context.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubernetesLeaderElectionConfiguration.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.kubeclient;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesHighAvailabilityOptions;
+
+import java.time.Duration;
+
+/**
+ * Configuration specific to {@link org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector}.
+ */
+public class KubernetesLeaderElectionConfiguration {
+
+	private final String clusterId;
+	private final String configMapName;
+	private final Duration leaseDuration;
+	private final Duration renewDeadline;
+	private final Duration retryPeriod;
+
+	private KubernetesLeaderElectionConfiguration(
+			String clusterId,
+			String configMapName,
+			Duration leaseDuration,
+			Duration renewDeadline,
+			Duration retryPeriod) {
+		this.clusterId = clusterId;
+		this.configMapName = configMapName;
+		this.leaseDuration = leaseDuration;
+		this.renewDeadline = renewDeadline;
+		this.retryPeriod = retryPeriod;
+	}
+
+	public String getClusterId() {
+		return clusterId;
+	}
+
+	public String getConfigMapName() {
+		return configMapName;
+	}
+
+	public Duration getLeaseDuration() {
+		return leaseDuration;
+	}
+
+	public Duration getRenewDeadline() {
+		return renewDeadline;
+	}
+
+	public Duration getRetryPeriod() {
+		return retryPeriod;
+	}
+
+	public static KubernetesLeaderElectionConfiguration fromConfiguration(String configMapName, Configuration config) {

Review comment:
       nit: I would invert the order of the parameters because the method is called `fromConfiguration`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",

Review comment:
       The log message could be `"Failed to update ConfigMap {} with data {} because of concurrent modifications. Trying again."`

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",

Review comment:
       If concurrent writes can happen, then we should not log on `warn` level. Debug log level should be fine.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {

Review comment:
       Similar to the AbstractLeaderElectionService, I am wondering whether composition wouldn't make more sense than inheritance here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	/** The executor to run external IO operations on. */
+	protected final Executor executor;

Review comment:
       let's call it `ioExecutor`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	/** The executor to run external IO operations on. */
+	protected final Executor executor;
+
+	/** The runtime configuration. */
+	protected final Configuration configuration;
+
+	/** Store for arbitrary blobs. */
+	private final BlobStoreService blobStoreService;
+
+	public AbstractHaServices(Executor executor, Configuration config, BlobStoreService blobStoreService) {
+
+		this.executor = checkNotNull(executor);
+		this.configuration = checkNotNull(config);
+		this.blobStoreService = blobStoreService;

Review comment:
       If the other parameters are checked, then we should also null check `blobStoreService`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});

Review comment:
       I think this pattern won't work. I'd suggest to use `exceptionally`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	/** The executor to run external IO operations on. */
+	protected final Executor executor;
+
+	/** The runtime configuration. */
+	protected final Configuration configuration;
+
+	/** Store for arbitrary blobs. */
+	private final BlobStoreService blobStoreService;
+
+	public AbstractHaServices(Executor executor, Configuration config, BlobStoreService blobStoreService) {
+
+		this.executor = checkNotNull(executor);
+		this.configuration = checkNotNull(config);
+		this.blobStoreService = blobStoreService;
+	}
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForResourceManager());
+	}
+
+	@Override
+	public LeaderRetrievalService getDispatcherLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForDispatcher());
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		return createLeaderRetrievalService(getLeaderNameForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress) {
+		return getJobManagerLeaderRetriever(jobID);
+	}
+
+	@Override
+	public LeaderRetrievalService getClusterRestEndpointLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForRestServer());
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForResourceManager());
+	}
+
+	@Override
+	public LeaderElectionService getDispatcherLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForDispatcher());
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		return createLeaderElectionService(getLeaderNameForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderElectionService getClusterRestEndpointLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForRestServer());
+	}
+
+	@Override
+	public BlobStore createBlobStore() {
+		return blobStoreService;
+	}
+
+	@Override
+	public void close() throws Exception {
+		Throwable exception = null;
+
+		try {
+			blobStoreService.close();
+		} catch (Throwable t) {
+			exception = t;
+		}
+
+		internalClose();
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(exception, "Could not properly close the " + getClass().getSimpleName());
+		}
+	}
+
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		logger.info("Close and clean up all data for {}.", getClass().getSimpleName());
+
+		Throwable exception = null;
+
+		try {
+			blobStoreService.closeAndCleanupAllData();
+		} catch (Throwable t) {
+			exception = t;
+		}
+
+		try {
+			internalCleanup();
+		} catch (Throwable t) {
+			exception = ExceptionUtils.firstOrSuppressed(t, exception);
+		}
+
+		internalClose();
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(
+				exception, "Could not properly close and clean up all data of high availability service.");
+		}
+		logger.info("Finished cleaning up the high availability data.");
+	}
+
+	/**
+	 * Create leader election service with specified leaderName.
+	 * @param leaderName ConfigMap name in Kubernetes or child node path in Zookeeper.
+	 *
+	 * @return Return LeaderElectionService using Zookeeper or Kubernetes.
+	 */
+	protected abstract LeaderElectionService createLeaderElectionService(String leaderName);
+
+	/**
+	 * Create leader retrieval service with specified leaderName.
+	 * @param leaderName ConfigMap name in Kubernetes or child node path in Zookeeper.
+	 *
+	 * @return Return LeaderRetrievalService using Zookeeper or Kubernetes.
+	 */
+	protected abstract LeaderRetrievalService createLeaderRetrievalService(String leaderName);
+
+	/**
+	 * Closes components which don't distinguish between close and closeAndCleanupAllData.
+	 */
+	protected abstract void internalClose() throws Exception;

Review comment:
       Would be good to state what happens if this method throws an `Exception`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	/** The executor to run external IO operations on. */
+	protected final Executor executor;
+
+	/** The runtime configuration. */
+	protected final Configuration configuration;
+
+	/** Store for arbitrary blobs. */
+	private final BlobStoreService blobStoreService;
+
+	public AbstractHaServices(Executor executor, Configuration config, BlobStoreService blobStoreService) {
+
+		this.executor = checkNotNull(executor);
+		this.configuration = checkNotNull(config);
+		this.blobStoreService = blobStoreService;
+	}
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForResourceManager());
+	}
+
+	@Override
+	public LeaderRetrievalService getDispatcherLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForDispatcher());
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		return createLeaderRetrievalService(getLeaderNameForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress) {
+		return getJobManagerLeaderRetriever(jobID);
+	}
+
+	@Override
+	public LeaderRetrievalService getClusterRestEndpointLeaderRetriever() {
+		return createLeaderRetrievalService(getLeaderNameForRestServer());
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForResourceManager());
+	}
+
+	@Override
+	public LeaderElectionService getDispatcherLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForDispatcher());
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		return createLeaderElectionService(getLeaderNameForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderElectionService getClusterRestEndpointLeaderElectionService() {
+		return createLeaderElectionService(getLeaderNameForRestServer());
+	}
+
+	@Override
+	public BlobStore createBlobStore() {
+		return blobStoreService;
+	}
+
+	@Override
+	public void close() throws Exception {
+		Throwable exception = null;
+
+		try {
+			blobStoreService.close();
+		} catch (Throwable t) {
+			exception = t;
+		}
+
+		internalClose();
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(exception, "Could not properly close the " + getClass().getSimpleName());
+		}
+	}
+
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		logger.info("Close and clean up all data for {}.", getClass().getSimpleName());
+
+		Throwable exception = null;
+
+		try {
+			blobStoreService.closeAndCleanupAllData();
+		} catch (Throwable t) {
+			exception = t;
+		}
+
+		try {
+			internalCleanup();
+		} catch (Throwable t) {
+			exception = ExceptionUtils.firstOrSuppressed(t, exception);
+		}
+
+		internalClose();
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(
+				exception, "Could not properly close and clean up all data of high availability service.");
+		}
+		logger.info("Finished cleaning up the high availability data.");
+	}
+
+	/**
+	 * Create leader election service with specified leaderName.
+	 * @param leaderName ConfigMap name in Kubernetes or child node path in Zookeeper.
+	 *
+	 * @return Return LeaderElectionService using Zookeeper or Kubernetes.
+	 */
+	protected abstract LeaderElectionService createLeaderElectionService(String leaderName);
+
+	/**
+	 * Create leader retrieval service with specified leaderName.
+	 * @param leaderName ConfigMap name in Kubernetes or child node path in Zookeeper.
+	 *
+	 * @return Return LeaderRetrievalService using Zookeeper or Kubernetes.
+	 */
+	protected abstract LeaderRetrievalService createLeaderRetrievalService(String leaderName);
+
+	/**
+	 * Closes components which don't distinguish between close and closeAndCleanupAllData.
+	 */
+	protected abstract void internalClose() throws Exception;
+
+	/**
+	 * Clean up the meta data in the distributed system(e.g. Zookeeper, Kubernetes ConfigMap).
+	 */
+	protected abstract void internalCleanup() throws Exception;

Review comment:
       Same here.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesConfigMapWatcher.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+
+import io.fabric8.kubernetes.api.model.ConfigMap;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Watcher for {@link ConfigMap}s in Kubernetes.

Review comment:
       `{@link ConfigMap ConfigMaps}` solves the problem of a plural here.

##########
File path: docs/_includes/generated/expert_high_availability_k8s_section.html
##########
@@ -0,0 +1,30 @@
+<table class="table table-bordered">
+    <thead>
+        <tr>
+            <th class="text-left" style="width: 20%">Key</th>
+            <th class="text-left" style="width: 15%">Default</th>
+            <th class="text-left" style="width: 10%">Type</th>
+            <th class="text-left" style="width: 55%">Description</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td><h5>high-availability.kubernetes.leader-election.lease-duration</h5></td>
+            <td style="word-wrap: break-word;">15 s</td>
+            <td>Duration</td>
+            <td>Define the lease duration for the Kubernetes leader election in ms. The leader will continuously renew its lease time to indicate its existence. And the followers will do a lease checking against the current time. "renewTime + leaseDuration &gt; now" means the leader is alive.</td>

Review comment:
       `in s` instead of `in ms`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						}
+						return updated;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),

Review comment:
       Does it make sense to retry this operation if the config map does not exist? Maybe we should stop the operation right after realizing the non-existence of the config map.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	protected volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	protected volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	protected volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+			internalStop();
+		}
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (logger.isDebugEnabled()) {
+			logger.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		if (hasLeadership()) {
+			// check if this is an old confirmation call
+			synchronized (lock) {
+				if (running) {
+					if (leaderSessionID.equals(this.issuedLeaderSessionID)) {
+						confirmLeaderInformation(leaderSessionID, leaderAddress);
+						writeLeaderInformation();
+					}
+				} else {
+					logger.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"LeaderElectionService has already been stopped.", leaderSessionID);
+				}
+			}
+		} else {
+			logger.warn("The leader session ID {} was confirmed even though the " +
+				"corresponding JobManager was not elected as the leader.", leaderSessionID);
+		}
+	}
+
+	@Override
+	public final boolean hasLeadership(@Nonnull UUID leaderSessionId) {
+		return hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID);

Review comment:
       `issuedLeaderSessionID` is accessed outside of `lock` scope.

##########
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:
       Maybe call this parameter `updateFunction`

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						}
+						return updated;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),

Review comment:
       `ConfigMap ... does not exist.`

##########
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 + "'}";
+	}

Review comment:
       For what do we need the `toString` method?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,69 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateConfigMap() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+		final Optional<KubernetesConfigMap> currentOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(currentOpt.isPresent(), is(true));
+		assertThat(currentOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	@Test
+	public void testDeleteConfigMapByLabels() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(TESTING_LABELS);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testDeleteConfigMapByName() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(
+			TESTING_CONFIG_MAP_NAME,
+			c -> {
+				c.getData().put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_NEW_VALUE);
+				return Optional.of(c);
+			}).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_NEW_VALUE));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMapWithEmptyResult() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker not pass and return empty result
+		this.flinkKubeClient.checkAndUpdateConfigMap(TESTING_CONFIG_MAP_NAME, c -> Optional.empty()).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	private KubernetesConfigMap buildTestingConfigMap() {
+		final Map<String, String> data = new HashMap<>();
+		data.put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_VALUE);
+		return new KubernetesConfigMap(new ConfigMapBuilder()
+			.withNewMetadata()
+			.withName(TESTING_CONFIG_MAP_NAME)
+			.withLabels(TESTING_LABELS)
+			.endMetadata()
+			.withData(data).build());
+	}

Review comment:
       Maybe add a test that concurrent writes will be resolved and don't overwrite each other.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();

Review comment:
       Ensuring the correct usage of synchronization patterns across sub classes is usually very hard and brittle because you never know which other sub class will be added later. The same applies a bit to exposing more or less all the fields of this class to the sub classes.
   
   Have you considered using composition instead of inheritance to achieve the goal of sharing the common logic? I think it should be feasible to have a `LeaderElectionService` which takes system dependent connector (Zk, K8s) which is responsible for the interaction with the external system. Similar to what Xintong did with the `ResourceManager` and the `ResourceManagerDrivers`.
   
   The benefit of using composition would be that we don't expose internal details of this class. Ideally, we would keep the synchronization logic all contained in this class.

##########
File path: docs/_includes/generated/expert_high_availability_k8s_section.html
##########
@@ -0,0 +1,30 @@
+<table class="table table-bordered">
+    <thead>
+        <tr>
+            <th class="text-left" style="width: 20%">Key</th>
+            <th class="text-left" style="width: 15%">Default</th>
+            <th class="text-left" style="width: 10%">Type</th>
+            <th class="text-left" style="width: 55%">Description</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td><h5>high-availability.kubernetes.leader-election.lease-duration</h5></td>
+            <td style="word-wrap: break-word;">15 s</td>
+            <td>Duration</td>
+            <td>Define the lease duration for the Kubernetes leader election in ms. The leader will continuously renew its lease time to indicate its existence. And the followers will do a lease checking against the current time. "renewTime + leaseDuration &gt; now" means the leader is alive.</td>
+        </tr>
+        <tr>
+            <td><h5>high-availability.kubernetes.leader-election.renew-deadline</h5></td>
+            <td style="word-wrap: break-word;">15 s</td>
+            <td>Duration</td>
+            <td>Defines the deadline when the leader tries to renew the lease in ms. The leader will give up its leadership if it cannot successfully renew the lease in the given time.</td>
+        </tr>
+        <tr>
+            <td><h5>high-availability.kubernetes.leader-election.retry-period</h5></td>
+            <td style="word-wrap: break-word;">5 s</td>
+            <td>Duration</td>
+            <td>Defines the pause between consecutive retries in ms. All the contenders, including the current leader and all other followers, periodically try to acquire/renew the leadership if possible at this interval.</td>

Review comment:
       `in s` instead of `in ms`.

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

Review comment:
       `@param configMapName configMapName specifies the name of the ConfigMap which shall be updated`

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+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.NAME_SEPARATOR;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * High availability service for Kubernetes.
+ */
+public class KubernetesHaServices extends AbstractHaServices {
+
+	private final String clusterId;
+
+	/** Kubernetes client. */
+	private final FlinkKubeClient kubeClient;
+
+	private static final String RESOURCE_MANAGER_NAME = "resourcemanager";
+
+	private static final String DISPATCHER_NAME = "dispatcher";
+
+	private static final String JOB_MANAGER_NAME = "jobmanager";
+
+	private static final String REST_SERVER_NAME = "restserver";
+
+	private static final String LEADER_SUFFIX = "leader";
+
+	KubernetesHaServices(
+			FlinkKubeClient kubeClient,
+			Executor executor,
+			Configuration config,
+			BlobStoreService blobStoreService) {
+
+		super(executor, config, blobStoreService);
+		this.kubeClient = checkNotNull(kubeClient);
+		this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID));
+	}
+
+	@Override
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
+		return new StandaloneCheckpointRecoveryFactory();
+	}
+
+	@Override
+	public JobGraphStore getJobGraphStore() {
+		return new StandaloneJobGraphStore();
+	}
+
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() {
+		return new StandaloneRunningJobsRegistry();
+	}
+
+	@Override
+	public LeaderElectionService createLeaderElectionService(String leaderName) {
+		return new KubernetesLeaderElectionService(
+			kubeClient,
+			executor,
+			KubernetesLeaderElectionConfiguration.fromConfiguration(leaderName, configuration));
+	}
+
+	@Override
+	public LeaderRetrievalService createLeaderRetrievalService(String leaderName) {
+		return new KubernetesLeaderRetrievalService(kubeClient, leaderName);
+	}
+
+	@Override
+	public void internalClose() throws Exception {
+		kubeClient.close();

Review comment:
       Due to this call, `KubeClientFactory.getFlinkKubeClient` must not return a singleton. If the `KubernetesHaServices` own the `kubeClient`, then we need to create separate clients.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */

Review comment:
       I think this JavaDoc should state the contracts of this class and what a user needs to pay attention to when writing a sub class, in particular since we expose the `lock` and most of its fields.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,69 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateConfigMap() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+		final Optional<KubernetesConfigMap> currentOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(currentOpt.isPresent(), is(true));
+		assertThat(currentOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	@Test
+	public void testDeleteConfigMapByLabels() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(TESTING_LABELS);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testDeleteConfigMapByName() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(
+			TESTING_CONFIG_MAP_NAME,
+			c -> {
+				c.getData().put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_NEW_VALUE);
+				return Optional.of(c);
+			}).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_NEW_VALUE));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMapWithEmptyResult() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker not pass and return empty result
+		this.flinkKubeClient.checkAndUpdateConfigMap(TESTING_CONFIG_MAP_NAME, c -> Optional.empty()).get();

Review comment:
       Maybe assert that this returns `false`.

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

Review comment:
       `@Nullable`

##########
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}.

Review comment:
       The configuration is an implementation detail of an implementation of this interface. I think it should not be mentioned in the JavaDocs of this interface. Instead one could say that implementations of this method will retry the operation. The number of retries is implementation specific, though.

##########
File path: docs/_includes/generated/expert_high_availability_k8s_section.html
##########
@@ -0,0 +1,30 @@
+<table class="table table-bordered">
+    <thead>
+        <tr>
+            <th class="text-left" style="width: 20%">Key</th>
+            <th class="text-left" style="width: 15%">Default</th>
+            <th class="text-left" style="width: 10%">Type</th>
+            <th class="text-left" style="width: 55%">Description</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td><h5>high-availability.kubernetes.leader-election.lease-duration</h5></td>
+            <td style="word-wrap: break-word;">15 s</td>
+            <td>Duration</td>
+            <td>Define the lease duration for the Kubernetes leader election in ms. The leader will continuously renew its lease time to indicate its existence. And the followers will do a lease checking against the current time. "renewTime + leaseDuration &gt; now" means the leader is alive.</td>
+        </tr>
+        <tr>
+            <td><h5>high-availability.kubernetes.leader-election.renew-deadline</h5></td>
+            <td style="word-wrap: break-word;">15 s</td>
+            <td>Duration</td>
+            <td>Defines the deadline when the leader tries to renew the lease in ms. The leader will give up its leadership if it cannot successfully renew the lease in the given time.</td>

Review comment:
       `in s` instead of `in ms`.

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

Review comment:
       nit: this does not really describe what the `configMap` parameter is.

##########
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);

Review comment:
       Can't it happen that we are removing the leader information of our successor here if this call is late? The next leader might already have written its leader information once this is executed.

##########
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);

Review comment:
       What if `leaderElector::run` fails? Should we assert that this won't throw an exception via `FutureUtils.assertNoException`?

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

Review comment:
       Maybe state that the returned future will be completed exceptionally if the ConfigMap already exists.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,69 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateConfigMap() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+		final Optional<KubernetesConfigMap> currentOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(currentOpt.isPresent(), is(true));
+		assertThat(currentOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	@Test
+	public void testDeleteConfigMapByLabels() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(TESTING_LABELS);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testDeleteConfigMapByName() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(
+			TESTING_CONFIG_MAP_NAME,
+			c -> {
+				c.getData().put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_NEW_VALUE);
+				return Optional.of(c);
+			}).get();

Review comment:
       Should we assert that this expression returns `true`?

##########
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
+	 *                      optional is empty, we will not do the update.

Review comment:
       I'd suggest to not align param descriptions because one changes the parameter name, then it will get quickly out of sync again.

##########
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);

Review comment:
       What is the meaning of a non-existent ConfigMap with name `configMapName`? Can this ever happen? Is this an invalid state where we should fail hard?

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

Review comment:
       Why are we setting `leaderContender` here?

##########
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) {

Review comment:
       Aren't we accessing these fields outside of the guard `lock` here?

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

Review comment:
       `@param name name of the ConfigMap to retrieve`

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

Review comment:
       Why are the JavaDocs referring to the `JobManagers`? Isn't this service also used for other leader elections?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalService.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+
+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;
+
+/**
+ * The counterpart to the {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionService}.
+ * This implementation of the {@link LeaderRetrievalService} retrieves the current leader which has
+ * been elected by the {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionService}.
+ * The leader address as well as the current leader session ID is retrieved from Kubernetes ConfigMap.
+ */
+class KubernetesLeaderRetrievalService implements LeaderRetrievalService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final String configMapName;
+
+	@GuardedBy("lock")
+	private volatile String lastLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile UUID lastLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile LeaderRetrievalListener leaderListener;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private KubernetesWatch kubernetesWatch;
+
+	KubernetesLeaderRetrievalService(FlinkKubeClient kubeClient, String configMapName) {
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client should not be null.");
+		this.configMapName = checkNotNull(configMapName, "ConfigMap name should not be null.");
+
+		this.leaderListener = null;
+		this.lastLeaderAddress = null;
+		this.lastLeaderSessionID = null;
+
+		running = false;
+	}
+
+	@Override
+	public void start(LeaderRetrievalListener listener) {
+		checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "KubernetesLeaderRetrievalService can " +
+			"only be started once.");
+
+		LOG.info("Starting {}.", this);
+
+		synchronized (lock) {
+			running = true;
+			leaderListener = listener;
+			kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		}
+	}
+
+	@Override
+	public void stop() {
+		LOG.info("Stopping {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			if (kubernetesWatch != null) {
+				kubernetesWatch.close();
+			}
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "KubernetesLeaderRetrievalService{configMapName='" + configMapName + "'}";
+	}
+
+	private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> {
+
+		@Override
+		public void onAdded(List<KubernetesConfigMap> configMaps) {
+			handleEvent(configMaps);
+		}
+
+		@Override
+		public void onModified(List<KubernetesConfigMap> configMaps) {
+			handleEvent(configMaps);
+		}
+
+		@Override
+		public void onDeleted(List<KubernetesConfigMap> configMaps) {
+			// Nothing to do since a new ConfigMap will be created if it is deleted externally.
+		}
+
+		@Override
+		public void onError(List<KubernetesConfigMap> configMaps) {
+			leaderListener.handleError(new Exception("Error while watching the ConfigMap " + configMapName));
+		}
+
+		@Override
+		public void handleFatalError(Throwable throwable) {
+			leaderListener.handleError(
+				new Exception("Fatal error while watching the ConfigMap " + configMapName, throwable));
+		}
+
+		private void handleEvent(List<KubernetesConfigMap> configMaps) {
+			synchronized (lock) {
+				if (running) {
+					configMaps.forEach(e -> {
+						if (e.getName().equals(configMapName)) {
+							final String leaderAddress = e.getData().get(LEADER_ADDRESS_KEY);
+							final String sessionID = e.getData().get(LEADER_SESSION_ID_KEY);
+							if (leaderAddress != null && sessionID != null) {
+								final UUID leaderSessionID = UUID.fromString(sessionID);
+								if (!(Objects.equals(leaderAddress, lastLeaderAddress) &&
+									Objects.equals(leaderSessionID, lastLeaderSessionID))) {
+									LOG.debug(
+										"New leader information: Leader={}, session ID={}.",
+										leaderAddress,
+										leaderSessionID);
+
+									lastLeaderAddress = leaderAddress;
+									lastLeaderSessionID = leaderSessionID;
+									leaderListener.notifyLeaderAddress(leaderAddress, leaderSessionID);
+								}
+							} else {
+								LOG.debug("Leader information was lost: The listener will be notified accordingly.");
+								leaderListener.notifyLeaderAddress(
+									leaderAddress, sessionID == null ? null : UUID.fromString(sessionID));
+							}

Review comment:
       I guess this logic could be factored out and shared between the Zk and K8s implementations.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	protected static final String LOCK_IDENTITY = UUID.randomUUID().toString();

Review comment:
       Won't it be a problem to have only one identity if we have multiple election services for the same type (e.g. multiple JobManagers running in the `MiniCluster`)?

##########
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) {

Review comment:
       How can it happen that the underlying configMap gets modified? Shouldn't only the leader be able to write new information by checking that the `LEADER_ANNOTATION_KEY` contains his `LOCK_IDENTITY` and only write if this is the case?

##########
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:
       What happens if the function throws an exception? I am asking because we are using a `FunctionWithException` here.

##########
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:
       What about integration tests for the `KubernetesLeaderElectionService` and the `KubernetesLeaderRetrievalService` with some K8s server?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	protected static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	protected static final String LEADER_ANNOTATION_KEY = "control-plane.alpha.kubernetes.io/leader";
+
+	public KubernetesLeaderElector(
+			NamespacedKubernetesClient kubernetesClient,
+			String namespace,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderCallbackHandler leaderCallbackHandler) {
+		super(kubernetesClient, new LeaderElectionConfigBuilder()
+			.withName(leaderConfig.getConfigMapName())
+			.withLeaseDuration(leaderConfig.getLeaseDuration())
+			.withLock(new ConfigMapLock(namespace, leaderConfig.getConfigMapName(), LOCK_IDENTITY))
+			.withRenewDeadline(leaderConfig.getRenewDeadline())
+			.withRetryPeriod(leaderConfig.getRetryPeriod())
+			.withLeaderCallbacks(new LeaderCallbacks(
+				leaderCallbackHandler::isLeader,
+				leaderCallbackHandler::notLeader,
+				newLeader -> LOG.info("New leader elected {}.", newLeader)
+			))
+			.build());
+		LOG.info("Create KubernetesLeaderElector {} with lock identity {}.",
+			leaderConfig.getConfigMapName(), LOCK_IDENTITY);
+	}
+
+	public boolean hasLeadership(KubernetesConfigMap configMap) {
+		final String leader = configMap.getAnnotations().get(LEADER_ANNOTATION_KEY);

Review comment:
       Could you explain in the JavaDocs, how the `LEADER_ANNOTATION_KEY` becomes an annotation of the `configMap`? And maybe add a bit of details which underlying K8s principles we are using here.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionServiceTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.apache.flink.runtime.leaderelection.TestingContender;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_KEY;
+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 KubernetesLeaderElectionService}.
+ */
+public class KubernetesLeaderElectionServiceTest extends KubernetesHighAvailabilityTestBase {
+
+	@Test
+	public void testKubernetesLeaderElection() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					assertThat(leaderElectionService.getLeaderSessionID(), is(contender.getLeaderSessionID()));
+					assertThat(configMapStore.size(), is(1));
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+
+					// Revoke leader
+					leaderController.set(false);
+					contender.waitForRevokeLeader(TIMEOUT);
+					assertThat(leaderElectionService.getLeaderSessionID(), nullValue());
+					assertThat(getLeaderConfigMap().getData().size(), is(0));
+				});
+		}};
+	}
+
+	@Test
+	public void testLeaderConfigMapDeletedExternally() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					final FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> callbackHandler =
+						configMapsAndDoCallbackFuture.get(TIMEOUT, TimeUnit.MILLISECONDS);
+					assertThat(callbackHandler, is(notNullValue()));
+
+					callbackHandler.onDeleted(Collections.singletonList(configMapStore.remove(LEADER_CONFIGMAP_NAME)));
+					// The ConfigMap should be created again.
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+				});
+		}};
+	}

Review comment:
       Is this something we want and have to support?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubeClientFactory.java
##########
@@ -71,7 +73,12 @@ public static FlinkKubeClient fromConfiguration(Configuration flinkConfig) {
 
 		final KubernetesClient client = new DefaultKubernetesClient(config);
 
-		return new Fabric8FlinkKubeClient(flinkConfig, client, KubeClientFactory::createThreadPoolForAsyncIO);
+		if (flinkKubeClient == null) {
+			flinkKubeClient = new Fabric8FlinkKubeClient(
+				flinkConfig, client, KubeClientFactory::createThreadPoolForAsyncIO);
+		}

Review comment:
       How many `flinkKubeClients` will be created when running Flink on K8s?

##########
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:
       How can it happen that ConfigMaps are being deleted?

##########
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);

Review comment:
       `FutureUtils.assertNoException`

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionServiceTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.apache.flink.runtime.leaderelection.TestingContender;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_KEY;
+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 KubernetesLeaderElectionService}.
+ */
+public class KubernetesLeaderElectionServiceTest extends KubernetesHighAvailabilityTestBase {
+
+	@Test
+	public void testKubernetesLeaderElection() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					assertThat(leaderElectionService.getLeaderSessionID(), is(contender.getLeaderSessionID()));
+					assertThat(configMapStore.size(), is(1));
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+
+					// Revoke leader
+					leaderController.set(false);
+					contender.waitForRevokeLeader(TIMEOUT);
+					assertThat(leaderElectionService.getLeaderSessionID(), nullValue());
+					assertThat(getLeaderConfigMap().getData().size(), is(0));
+				});
+		}};
+	}
+
+	@Test
+	public void testLeaderConfigMapDeletedExternally() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					final FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> callbackHandler =
+						configMapsAndDoCallbackFuture.get(TIMEOUT, TimeUnit.MILLISECONDS);
+					assertThat(callbackHandler, is(notNullValue()));
+
+					callbackHandler.onDeleted(Collections.singletonList(configMapStore.remove(LEADER_CONFIGMAP_NAME)));
+					// The ConfigMap should be created again.
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+				});
+		}};
+	}
+
+	@Test
+	public void testLeaderConfigMapUpdatedExternally() throws Exception {
+		new Context() {{
+			runTestAndGrantLeadershipToContender(
+				() -> {
+					final FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> callbackHandler =
+						configMapsAndDoCallbackFuture.get(TIMEOUT, TimeUnit.MILLISECONDS);
+					assertThat(callbackHandler, is(notNullValue()));
+					// Update ConfigMap with wrong data
+					getLeaderConfigMap().getData().put(Constants.LEADER_ADDRESS_KEY, "wrong data");
+					callbackHandler.onModified(Collections.singletonList(configMapStore.get(LEADER_CONFIGMAP_NAME)));
+					// The ConfigMap should be corrected
+					assertThat(getLeaderConfigMap().getData().get(Constants.LEADER_ADDRESS_KEY), is(LEADER_URL));
+				});
+		}};
+	}
+
+	/**
+	 * Start multiple leaders, if current leader dies, a new one could take over and update the leader
+	 * information successfully.
+	 */
+	@Test
+	public void testMultipleLeaders() throws Exception {
+		final int leaderNum = 3;
+		final AtomicBoolean[] leaderController = new AtomicBoolean[leaderNum];
+		final TestingContender[] contenders = new TestingContender[leaderNum];
+		final KubernetesLeaderElectionService[] leaderElectionServices = new KubernetesLeaderElectionService[leaderNum];
+		for (int i = 0; i < leaderNum; i++) {
+			leaderController[i] = new AtomicBoolean(false);
+			leaderElectionServices[i] = new Context().createLeaderElectionService(leaderController[i]);
+			contenders[i] = new TestingContender(getLeaderUrl(i), leaderElectionServices[i]);
+			leaderElectionServices[i].start(contenders[i]);
+		}
+		leaderController[0].set(true);
+		contenders[0].waitForLeader(TIMEOUT);
+		assertThat(
+			configMapStore.get(LEADER_CONFIGMAP_NAME).getData().get(Constants.LEADER_ADDRESS_KEY),
+			is(getLeaderUrl(0)));
+		// Leader 0 died
+		leaderController[0].set(false);
+		contenders[0].waitForRevokeLeader(TIMEOUT);
+		// Leader 2 try to acquire
+		leaderController[2].set(true);
+		contenders[2].waitForLeader(TIMEOUT);
+		assertThat(
+			configMapStore.get(LEADER_CONFIGMAP_NAME).getData().get(Constants.LEADER_ADDRESS_KEY),
+			is(getLeaderUrl(2)));
+		for (int i = 0; i < leaderNum; i++) {
+			leaderElectionServices[i].stop();
+		}
+	}

Review comment:
       Shouldn't we also have a test were we have multiple leaders concurrently trying to obtain the leadership using the proper `KubernetesLeaderElector`?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {

Review comment:
       I think we are missing tests for this class.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514061417



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);

Review comment:
       Yes, `leaderElector::run` is a blocking call. It will block the thread until lost the leadership. So it will make the `ioExecutor` should be configured big. Based on this, I will introduce a dedicated single thread pool for `KubernetesLeaderElector`. And then we could support the `stop` method to shutdown the thread pool.
   
   Also the `leaderElectorRunFuture` will be unnecessary.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513212153



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalDriver.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link ZooKeeperLeaderElectionDriver}.
+ * {@link LeaderRetrievalService} implementation for Zookeeper. It retrieves the current leader which has
+ * been elected by the {@link ZooKeeperLeaderElectionDriver}.
+ * The leader address as well as the current leader session ID is retrieved from ZooKeeper.
+ */
+public class ZooKeeperLeaderRetrievalDriver implements LeaderRetrievalDriver, NodeCacheListener, UnhandledErrorListener {
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderRetrievalDriver.class);
+
+	/** Connection to the used ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe to watch changes of a specific ZooKeeper node. */
+	private final NodeCache cache;
+
+	private final String retrievalPath;
+
+	private final ConnectionStateListener connectionStateListener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderRetrievalEventHandler leaderRetrievalEventHandler;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a leader retrieval service which uses ZooKeeper to retrieve the leader information.
+	 *
+	 * @param client Client which constitutes the connection to the ZooKeeper quorum
+	 * @param retrievalPath Path of the ZooKeeper node which contains the leader information
+	 * @param leaderRetrievalEventHandler handler to notify the leader changes.
+	 */
+	public ZooKeeperLeaderRetrievalDriver(
+			CuratorFramework client,
+			String retrievalPath,
+			LeaderRetrievalEventHandler leaderRetrievalEventHandler) throws Exception {
+		this.client = checkNotNull(client, "CuratorFramework client");
+		this.cache = new NodeCache(client, retrievalPath);
+		this.retrievalPath = checkNotNull(retrievalPath);
+
+		this.leaderRetrievalEventHandler = checkNotNull(leaderRetrievalEventHandler);
+
+		client.getUnhandledErrorListenable().addListener(this);
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(connectionStateListener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception {
+		if (!running) {
+			return;
+		}
+
+		running = false;
+
+		LOG.info("Closing {}.", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+		client.getConnectionStateListenable().removeListener(connectionStateListener);
+
+		try {
+			cache.close();
+		} catch (IOException e) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderRetrievalDriver.", e);
+		}
+	}
+
+	@Override
+	public void nodeChanged() {
+		retrieveLeaderInformationFromZooKeeper();
+	}
+
+	private void retrieveLeaderInformationFromZooKeeper() {
+		if (running) {
+			try {
+				LOG.debug("Leader node has changed.");
+
+				ChildData childData = cache.getCurrentData();
+
+				String leaderAddress;
+				UUID leaderSessionID;
+
+				if (childData == null) {
+					leaderAddress = null;
+					leaderSessionID = null;
+				} else {
+					byte[] data = childData.getData();
+
+					if (data == null || data.length == 0) {
+						leaderAddress = null;
+						leaderSessionID = null;
+					} else {
+						ByteArrayInputStream bais = new ByteArrayInputStream(data);
+						ObjectInputStream ois = new ObjectInputStream(bais);
+
+						leaderAddress = ois.readUTF();
+						leaderSessionID = (UUID) ois.readObject();
+					}
+				}
+
+				leaderRetrievalEventHandler.notifyLeaderAddress(new LeaderInformation(leaderSessionID, leaderAddress));
+			} catch (Exception e) {
+				leaderRetrievalEventHandler.handleError(new Exception("Could not handle node changed event.", e));
+				ExceptionUtils.checkInterrupted(e);
+			}
+		} else {
+			LOG.debug("Ignoring node change notification since the service has already been stopped.");
+		}
+	}
+
+	private void handleStateChange(ConnectionState newState) {
+		switch (newState) {
+			case CONNECTED:
+				LOG.debug("Connected to ZooKeeper quorum. Leader retrieval can start.");
+				break;
+			case SUSPENDED:
+				LOG.warn("Connection to ZooKeeper suspended. Can no longer retrieve the leader from " +
+					"ZooKeeper.");
+				leaderRetrievalEventHandler.notifyLeaderAddress(LeaderInformation.empty());

Review comment:
       Yes. We need to check the `running` state.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 7f34f6f31596832223b1cf2010bed1c47f7502ea Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511) 
   * 6a2602f39684ca0005350787b1c8051815acecdc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508994663



##########
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);

Review comment:
       If we agree to not handle the externally deletion/update, then we should fail hardly if the ConfigMap not exists.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507625204



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionService.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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 java.util.List;
+import java.util.Map;
+import java.util.Objects;
+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.watchConfigMapsAndDoCallback(configMapName, new ConfigMapCallbackHandlerImpl());
+	}
+
+	@Override
+	public void internalStop() {
+		if (kubernetesWatch != null) {
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	protected void writeLeaderInformation() {
+		updateConfigMap(configMapName);
+	}
+
+	@Override
+	protected boolean checkLeaderLatch() {
+		return kubeClient.getConfigMap(configMapName)
+			.map(configMap -> KubernetesUtils.getLeaderChecker().test(configMap))
+			.orElse(false);
+	}
+
+	@Override
+	public String toString() {
+		return "KubernetesLeaderElectionService{configMapName='" + configMapName + "'}";
+	}
+
+	private void updateConfigMap(String configMapName) {
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				KubernetesUtils.getLeaderChecker(),
+				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 configMap;
+				}).get();
+		} catch (Exception e) {
+			leaderContender.handleError(new Exception("Could not update ConfigMap " + configMapName, e));
+		}
+	}
+
+	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,
+					KubernetesUtils.getLeaderChecker(),
+					configMap -> {
+						configMap.getData().remove(LEADER_ADDRESS_KEY);
+						configMap.getData().remove(LEADER_SESSION_ID_KEY);
+						return 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 (checkLeaderLatch()) {
+				configMaps.forEach(configMap -> {
+					if (isLeaderChanged(configMap)) {
+						// the data field does not correspond to the expected leader information
+						if (logger.isDebugEnabled()) {
+							logger.debug("Correcting leader information in {} by {}.",
+								configMapName, leaderContender.getDescription());
+						}
+						updateConfigMap(configMap.getName());
+					}
+				});
+			}
+		}

Review comment:
       We have a chance that we receive the modified event before `confirmLeadership`. So I will add a check for `confirmedLeaderSessionID` and `confirmedLeaderAddress` should not be null.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507557947



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesHighAvailabilityOptions.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to Kubernetes high-availability settings.
+ * All the HA information relevant for a specific component will be stored in a single ConfigMap.
+ * For example, the Dispatcher's ConfigMap would then contain the current leader, the running jobs
+ * and the pointers to the persisted JobGraphs.
+ * The JobManager's ConfigMap would then contain the current leader, the pointers to the checkpoints
+ * and the checkpoint ID counter.
+ *
+ * <p>The ConfigMap name will be created with the following pattern.
+ * e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader

Review comment:
       The ConfigMap name will be created with the pattern '\<clusterId>-\<componentName>-leader'. Given that the cluster id is configured to "k8s-ha-app1", then we could get the following ConfigMap names. 
   e.g. k8s-ha-app1-restserver-leader, k8s-ha-app1-00000000000000000000000000000000-jobmanager-leader




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708923611


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit c32eeb3b2a45a5bb349394f9651c2e54b80bbfc3 (Thu Oct 15 06:14:53 UTC 2020)
   
   **Warnings:**
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-19542).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508988440



##########
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:
       If we have an exception in the `updateFunction`, we convert it to the unchecked exception and retry again. Since a new retry attempt will get the latest ConfigMap, the `updateFunction` could then execute successfully.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509891311



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	protected static final String LOCK_IDENTITY = UUID.randomUUID().toString();

Review comment:
       Maybe I am not making myself clear. The `lockIdentity` will be a non-static field in `KubernetesHaServices` after refactor, which means `KubernetesHaServices` will have a dedicated lock identity for all the components above. But different instances will have different identities.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514059389



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// 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) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not update ConfigMap " + configMapName, e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		final Optional<KubernetesConfigMap> configMapOpt = kubeClient.getConfigMap(configMapName);
+		if (configMapOpt.isPresent()) {
+			return KubernetesLeaderElector.hasLeadership(configMapOpt.get(), lockIdentity);
+		} else {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("ConfigMap " + configMapName + "does not exist.", null));
+			return false;
+		}
+	}
+
+	private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+		@Override
+		public void isLeader() {
+			synchronized (lock) {
+				if (running) {
+					leaderElectionEventHandler.onGrantLeadership();
+				}
+			}
+		}
+
+		@Override
+		public void notLeader() {
+			synchronized (lock) {
+				if (running) {
+					// Clear the leader information in ConfigMap
+					try {
+						kubeClient.checkAndUpdateConfigMap(
+							configMapName,
+							configMap -> {
+								if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+									configMap.getData().remove(LEADER_ADDRESS_KEY);
+									configMap.getData().remove(LEADER_SESSION_ID_KEY);
+									return Optional.of(configMap);
+								}
+								return Optional.empty();
+							}

Review comment:
       If we make the `writeLeaderInformation()` could write empty `LeaderInformation`, then moving this piece of codes is reasonable.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-716414850


   @tillrohrmann I have address the comments you left and pushed again. Compared with previous PR, you could find the following changes.
   
   * Introduce `LeaderElectionEventHandler` and `LeaderRetrievalEventHandler`. They define the contracts between `LeaderElectionDriver`/`LeaderRetrievalDriver` and contenders/listeners.
   * Make some `ZooKeeperLeaderElection` tests could directly work with driver
   * Add test for `DefaultLeaderElectionService`
   * Use driver(not instantiate `DefaultLeaderElectionService`) for the HA unit tests. Then the tests are more purposeful.
   * Add ITCases for Kubernetes complicated test case, concurrent leader election, concurrent `checkAndUpdateConfigMap`


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r510976264



##########
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:
       Hmm, how would this pattern work with concurrent modifications? Assume that the following happens: We get the `configMap`, discard the `stateHandle` and then return the updated `configMap` w/o the state handle key. Now for some reason `internalClient.createOrReplace` fails and the whole operation will be retried. Now when the `function` is re-executed it will fail right away because `stateHandle` can no longer be discarded but it is still in the `configMap` (because the update failed).
   
   I guess what I want to say is that we should not discard state which has not been removed from the `configMap`. Hence, it would be good if we could support an API where we first update the `configMap` and only do an operation if the update has succeeded.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509848812



##########
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:
       Yes. `KubernetesLeaderElector ` is responsible for creating the leader ConfigMap. And it could recreate it again when trying to renew the lock. However, the data of new ConfigMap is empty(just the annotation is kept). Then our running jobs, job graphs will be lost. That's why I try to create a same ConfigMap in `onDeleted` callback.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508497476



##########
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)) {

Review comment:
       Yes. Actually, we should only receive the watched ConfigMap event. Adding a `assert` here make more senses.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514290977



##########
File path: flink-end-to-end-tests/test-scripts/test_kubernetes_itcases.sh
##########
@@ -0,0 +1,30 @@
+#!/usr/bin/env bash
+################################################################################
+# 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.
+################################################################################
+
+source "$(dirname "$0")"/common_kubernetes.sh
+
+start_kubernetes
+
+# Set the ITCASE_KUBECONFIG environment since it is required to run the ITCases
+export ITCASE_KUBECONFIG=~/.kube/config
+
+cd $END_TO_END_DIR/../flink-kubernetes

Review comment:
       Do you mean we start the minikube in the `KubernetesResource`? Or we assume that we have an prepared minikube. Currently, we are using this script to start the minikube and run tests.
   
   If you already have a minikube running on your machine, set the `ITCASE_KUBECONFIG` environment, then we could directly run the test in IDE or maven.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507380643



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -106,13 +110,37 @@ public static String getDeploymentName(String clusterId) {
 	 * @return Task manager labels.
 	 */
 	public static Map<String, String> getTaskManagerLabels(String clusterId) {
-		final Map<String, String> labels = new HashMap<>();
-		labels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE);
-		labels.put(Constants.LABEL_APP_KEY, clusterId);
+		final Map<String, String> labels = new HashMap<>(getCommonLabels(clusterId));
 		labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_TASK_MANAGER);
 		return Collections.unmodifiableMap(labels);
 	}
 
+	/**
+	 * Get the common labels for Flink native clusters. All the Kubernetes resources will be set with these labels.
+	 *
+	 * @param clusterId cluster id
+	 * @return Return common labels map
+	 */
+	public static Map<String, String> getCommonLabels(String clusterId) {
+		Map<String, String> commonLabels = new HashMap<>();
+		commonLabels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE);
+		commonLabels.put(Constants.LABEL_APP_KEY, clusterId);
+
+		return Collections.unmodifiableMap(commonLabels);

Review comment:
       Yes. There's no harm to return a modifiable in the `KubernetesUtils#getCommonLabels`. For `KubernetesUtils#getTaskManagerLabels` and `KubernetesUtils#getConfigMapLabels`, we do not want the returned map to be updated. So we still return a unmodifiable Map.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507384035



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java
##########
@@ -80,4 +84,14 @@
 	public static final String RESTART_POLICY_OF_NEVER = "Never";
 
 	public static final String NATIVE_KUBERNETES_COMMAND = "native-k8s";
+
+	// Constants for Kubernetes high availability
+	public static final String LEADER_ADDRESS_KEY = "address";
+	public static final String LEADER_SESSION_ID_KEY = "sessionId";
+	public static final String CHECKPOINT_COUNTER_KEY = "counter";
+	public static final String RUNNING_JOBS_REGISTRY_KEY_PREFIX = "runningJobsRegistry";
+	public static final String JOB_GRAPH_STORE_KEY_PREFIX = "jobGraph";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();

Review comment:
       Moving the `LOCK_IDENTITY` to the `KubernetesLeaderElectionService` makes sense to me.
   However, I am not sure why do we need to make it non static. Actually, I want the lock identity to be same for different leader election service(e.g. Dispatcher, ResourceManager, etc.).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508488975



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServices.java
##########
@@ -102,146 +95,73 @@
 	/** The ZooKeeper client to use. */
 	private final CuratorFramework client;
 
-	/** The executor to run ZooKeeper callbacks on. */
-	private final Executor executor;
-
-	/** The runtime configuration. */
-	private final Configuration configuration;
-
-	/** The zookeeper based running jobs registry. */
-	private final RunningJobsRegistry runningJobsRegistry;
-
-	/** Store for arbitrary blobs. */
-	private final BlobStoreService blobStoreService;
-
 	public ZooKeeperHaServices(
 			CuratorFramework client,
 			Executor executor,
 			Configuration configuration,
 			BlobStoreService blobStoreService) {
+		super(executor, configuration, blobStoreService);
 		this.client = checkNotNull(client);
-		this.executor = checkNotNull(executor);
-		this.configuration = checkNotNull(configuration);
-		this.runningJobsRegistry = new ZooKeeperRunningJobsRegistry(client, configuration);
-
-		this.blobStoreService = checkNotNull(blobStoreService);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Services
-	// ------------------------------------------------------------------------
-
-	@Override
-	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
-	}
-
-	@Override
-	public LeaderRetrievalService getDispatcherLeaderRetriever() {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, DISPATCHER_LEADER_PATH);
 	}
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathForJobManager(jobID));
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
+		return new ZooKeeperCheckpointRecoveryFactory(client, configuration, executor);
 	}
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress) {
-		return getJobManagerLeaderRetriever(jobID);
+	public JobGraphStore getJobGraphStore() throws Exception {
+		return ZooKeeperUtils.createJobGraphs(client, configuration);
 	}
 
 	@Override
-	public LeaderRetrievalService getClusterRestEndpointLeaderRetriever() {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, REST_SERVER_LEADER_PATH);
+	public RunningJobsRegistry getRunningJobsRegistry() {
+		return new ZooKeeperRunningJobsRegistry(client, configuration);

Review comment:
       I will update this part and keep the old behavior.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509294434



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kubeclient.resources;
+
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+
+import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderCallbacks;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElectionConfigBuilder;
+import io.fabric8.kubernetes.client.extended.leaderelection.LeaderElector;
+import io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.ConfigMapLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+
+/**
+ * Represent {@link KubernetesLeaderElector} in kubernetes. {@link LeaderElector#run()} is a blocking call. It should be
+ *  run in the IO executor, not the main thread. The lifecycle is bound to single leader election. Once the leadership
+ * is revoked, as well as the {@link LeaderCallbackHandler#notLeader()} is called, the {@link LeaderElector#run()} will
+ * finish. To start another round of election, we need to trigger again.
+ */
+public class KubernetesLeaderElector extends LeaderElector<NamespacedKubernetesClient> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElector.class);
+	protected static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	protected static final String LEADER_ANNOTATION_KEY = "control-plane.alpha.kubernetes.io/leader";
+
+	public KubernetesLeaderElector(
+			NamespacedKubernetesClient kubernetesClient,
+			String namespace,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderCallbackHandler leaderCallbackHandler) {
+		super(kubernetesClient, new LeaderElectionConfigBuilder()
+			.withName(leaderConfig.getConfigMapName())
+			.withLeaseDuration(leaderConfig.getLeaseDuration())
+			.withLock(new ConfigMapLock(namespace, leaderConfig.getConfigMapName(), LOCK_IDENTITY))
+			.withRenewDeadline(leaderConfig.getRenewDeadline())
+			.withRetryPeriod(leaderConfig.getRetryPeriod())
+			.withLeaderCallbacks(new LeaderCallbacks(
+				leaderCallbackHandler::isLeader,
+				leaderCallbackHandler::notLeader,
+				newLeader -> LOG.info("New leader elected {}.", newLeader)
+			))
+			.build());
+		LOG.info("Create KubernetesLeaderElector {} with lock identity {}.",
+			leaderConfig.getConfigMapName(), LOCK_IDENTITY);
+	}
+
+	public boolean hasLeadership(KubernetesConfigMap configMap) {
+		final String leader = configMap.getAnnotations().get(LEADER_ANNOTATION_KEY);

Review comment:
       I think your suggestion makes sense. I will add more description about where the annotation comes from and how it looks like.
   
   ```
     annotations:
       control-plane.alpha.kubernetes.io/leader: '{"holderIdentity":"623e39fb-70c3-44f1-811f-561ec4a28d75","leaseDuration":15.000000000,"acquireTime":"2020-10-20T04:06:31.431000Z","renewTime":"2020-10-21T13:42:46.843000Z","leaderTransitions":24195}'
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509017125



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -103,6 +121,52 @@ public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, WatchC
 		return watchPodsAndDoCallbackFunction.apply(labels, podCallbackHandler);
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		configMapStore.putIfAbsent(configMap.getName(), configMap);
+		return CompletableFuture.completedFuture(null);
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final KubernetesConfigMap configMap = configMapStore.get(name);
+		if (configMap == null) {
+			return Optional.empty();
+		}
+		return Optional.of(new MockKubernetesConfigMap(configMap.getName(), new HashMap<>(configMap.getData())));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			Predicate<KubernetesConfigMap> checker,
+			FunctionWithException<KubernetesConfigMap, KubernetesConfigMap, ?> function) {
+		return getConfigMap(configMapName).map(FunctionUtils.uncheckedFunction(
+			configMap -> {
+				final boolean shouldUpdate = checker.test(configMap);
+				if (shouldUpdate) {
+					configMapStore.put(configMap.getName(), function.apply(configMap));
+				}
+				return CompletableFuture.completedFuture(shouldUpdate);
+			}))
+			.orElseThrow(() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists."));
+	}

Review comment:
       Hmm. I know your concerns now. Then I will try to use the `Function`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514043165



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);

Review comment:
       Both for ZooKeeper and Kubernetes, it could happen. But it seems that the old behavior is also like this and we do not find a critical issue.
   Maybe we do not need to check the `running` in the `LeaderElectionDriver` when calling the callbacks. Since when the driver is closed, all the callbacks will be ignored.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509856352



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,71 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		final String configMapName = configMap.getName();
+		return CompletableFuture.runAsync(
+			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
+			kubeClientExecutorService)
+			.whenComplete((ignored, throwable) -> {
+				if (throwable != null) {
+					throw new FlinkRuntimeException("Failed to create ConfigMap " + configMapName, throwable);
+				}
+			});
+	}
+
+	@Override
+	public Optional<KubernetesConfigMap> getConfigMap(String name) {
+		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
+		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
+	}
+
+	@Override
+	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
+			String configMapName,
+			FunctionWithException<KubernetesConfigMap, Optional<KubernetesConfigMap>, ?> function) {
+		return FutureUtils.retry(
+			() -> CompletableFuture.supplyAsync(
+				() -> getConfigMap(configMapName)
+					.map(FunctionUtils.uncheckedFunction(configMap -> {
+						final boolean updated = function.apply(configMap).map(
+							updatedConfigMap -> {
+								this.internalClient.configMaps()
+									.inNamespace(namespace)
+									.createOrReplace(updatedConfigMap.getInternalResource());
+								return true;
+							}).orElse(false);
+						if (!updated) {
+							LOG.warn("Trying to update ConfigMap {} to {} without checking pass, ignoring.",
+								configMap.getName(), configMap.getData());
+						}
+						return updated;
+					}))
+					.orElseThrow(
+						() -> new FlinkRuntimeException("ConfigMap " + configMapName + " not exists.")),

Review comment:
       What I mean is the ConfigMap could be created in the `KubernetesLeaderElectionService#Watcher`. So even the first we get a `Optional.empty()`, we could a get correct ConfigMap by retrying.
   
   I will add a two tests here.
   * ConfigMap always does not exists and retry failed
   * ConfigMap exists at the very begging and retry successfully




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508505644



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalService.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+
+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;
+
+/**
+ * The counterpart to the {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionService}.
+ * This implementation of the {@link LeaderRetrievalService} retrieves the current leader which has
+ * been elected by the {@link org.apache.flink.kubernetes.highavailability.KubernetesLeaderElectionService}.
+ * The leader address as well as the current leader session ID is retrieved from Kubernetes ConfigMap.
+ */
+class KubernetesLeaderRetrievalService implements LeaderRetrievalService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final String configMapName;
+
+	@GuardedBy("lock")
+	private volatile String lastLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile UUID lastLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile LeaderRetrievalListener leaderListener;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	private KubernetesWatch kubernetesWatch;
+
+	KubernetesLeaderRetrievalService(FlinkKubeClient kubeClient, String configMapName) {
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client should not be null.");
+		this.configMapName = checkNotNull(configMapName, "ConfigMap name should not be null.");
+
+		this.leaderListener = null;
+		this.lastLeaderAddress = null;
+		this.lastLeaderSessionID = null;
+
+		running = false;
+	}
+
+	@Override
+	public void start(LeaderRetrievalListener listener) {
+		checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "KubernetesLeaderRetrievalService can " +
+			"only be started once.");
+
+		LOG.info("Starting {}.", this);
+
+		synchronized (lock) {
+			running = true;
+			leaderListener = listener;
+			kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		}
+	}
+
+	@Override
+	public void stop() {
+		LOG.info("Stopping {}.", this);
+
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			if (kubernetesWatch != null) {
+				kubernetesWatch.close();
+			}
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "KubernetesLeaderRetrievalService{configMapName='" + configMapName + "'}";
+	}
+
+	private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> {
+
+		@Override
+		public void onAdded(List<KubernetesConfigMap> configMaps) {
+			handleEvent(configMaps);
+		}
+
+		@Override
+		public void onModified(List<KubernetesConfigMap> configMaps) {
+			handleEvent(configMaps);
+		}
+
+		@Override
+		public void onDeleted(List<KubernetesConfigMap> configMaps) {
+			// Nothing to do since a new ConfigMap will be created if it is deleted externally.
+		}
+
+		@Override
+		public void onError(List<KubernetesConfigMap> configMaps) {
+			leaderListener.handleError(new Exception("Error while watching the ConfigMap " + configMapName));
+		}
+
+		@Override
+		public void handleFatalError(Throwable throwable) {
+			leaderListener.handleError(
+				new Exception("Fatal error while watching the ConfigMap " + configMapName, throwable));
+		}
+
+		private void handleEvent(List<KubernetesConfigMap> configMaps) {
+			synchronized (lock) {
+				if (running) {
+					configMaps.forEach(e -> {
+						if (e.getName().equals(configMapName)) {

Review comment:
       Yes.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509045121



##########
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);

Review comment:
       I think we should condition every update call on the value of the current leader. A process should only be allowed to update the ConfigMap if his identifier is the value of the leader key.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * aad1ba94672d89de5a4fc814297f0c40e3552db0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-717892502


   @tillrohrmann I have addressed the comments you left and force pushed. They are all promising changes.
   
   You could continue the review :)


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 7ae53916e13b43e94621217d7a354a672077e0ab Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285) 
   * aad1ba94672d89de5a4fc814297f0c40e3552db0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507371480



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,68 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		return CompletableFuture.runAsync(
+			() -> {
+				if (!getConfigMap(configMap.getName()).isPresent()) {
+					this.internalClient.configMaps().create(configMap.getInternalResource());
+				}
+			},
+			kubeClientExecutorService);

Review comment:
       Yes. This is already in the plan. I have left a comments on the FLINK-19037. I have created a new ticket FLINK-19700 to follow this.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -104,6 +107,67 @@ KubernetesWatch watchPodsAndDoCallback(
 		Map<String, String> labels,
 		WatchCallbackHandler<KubernetesPod> podCallbackHandler);
 
+	/**
+	 * Create the ConfigMap with specified content. If the ConfigMap already exists, nothing will happen.
+	 *
+	 * @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 empty if the ConfigMap does not exist.
+	 */
+	Optional<KubernetesConfigMap> getConfigMap(String name);
+
+	/**
+	 * Update an existing ConfigMap with the data.
+	 *
+	 * @param configMapName ConfigMap to be replaced with. 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_MAX_RETRY_ATTEMPTS}.
+	 * @param checker       Only the checker return true, the ConfigMap will be updated.
+	 * @param function      The obtained ConfigMap will be applied to this function and get a new one to replace.
+	 *
+	 * @return Return the ConfigMap update future.
+	 */

Review comment:
       Make sense.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509276361



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,69 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateConfigMap() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+		final Optional<KubernetesConfigMap> currentOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(currentOpt.isPresent(), is(true));
+		assertThat(currentOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	@Test
+	public void testDeleteConfigMapByLabels() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(TESTING_LABELS);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testDeleteConfigMapByName() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(
+			TESTING_CONFIG_MAP_NAME,
+			c -> {
+				c.getData().put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_NEW_VALUE);
+				return Optional.of(c);
+			}).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_NEW_VALUE));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMapWithEmptyResult() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker not pass and return empty result
+		this.flinkKubeClient.checkAndUpdateConfigMap(TESTING_CONFIG_MAP_NAME, c -> Optional.empty()).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	private KubernetesConfigMap buildTestingConfigMap() {
+		final Map<String, String> data = new HashMap<>();
+		data.put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_VALUE);
+		return new KubernetesConfigMap(new ConfigMapBuilder()
+			.withNewMetadata()
+			.withName(TESTING_CONFIG_MAP_NAME)
+			.withLabels(TESTING_LABELS)
+			.endMetadata()
+			.withData(data).build());
+	}

Review comment:
       I will add a test for creating failed and an exception should be thrown.
   
   The reason why I do not test the concurrent writes is that we are using `KubernetesMockServer`. Actually, it could not support such semantic, the resource version is always null. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8720",
       "triggerID" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753",
       "triggerID" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884",
       "triggerID" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8956",
       "triggerID" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * ed02e962a354185fb37eaedbdfee217d691bd16b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884) 
   * e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8956) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8720",
       "triggerID" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753",
       "triggerID" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884",
       "triggerID" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8956",
       "triggerID" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8988",
       "triggerID" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce1452d50dcad0cd1e6decb42e26c3c352ee8d67",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9013",
       "triggerID" : "ce1452d50dcad0cd1e6decb42e26c3c352ee8d67",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * ce1452d50dcad0cd1e6decb42e26c3c352ee8d67 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9013) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-712548080


   @xintongsong Thanks for your careful review. I have addressed most of the comments except for some open questions. Please have a look and share your thoughts about them.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r512791010



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
##########
@@ -140,15 +161,22 @@
 						if (throwable instanceof CancellationException) {
 							resultFuture.completeExceptionally(new RetryException("Operation future was cancelled.", throwable));
 						} else {
-							if (retries > 0) {
-								retryOperation(
-									resultFuture,
-									operation,
-									retries - 1,
-									executor);
+							throwable = ExceptionUtils.stripExecutionException(throwable);
+							if (!retryPredicate.test(throwable)) {
+								resultFuture.completeExceptionally(throwable);

Review comment:
       Maybe fail with `RetryException("Stopped retrying the operation because the error is not retriable.", throwable)`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionDriverFactory.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.leaderelection;
+
+/**
+ * Factory for creating {@link LeaderElectionDriver} with different implementation.
+ */
+public interface LeaderElectionDriverFactory {
+
+	/**
+	 * Create a specific {@link LeaderElectionDriver} and start the necessary services. For example, LeaderLatch
+	 * and NodeCache in Zookeeper, KubernetesLeaderElector and ConfigMap watcher in Kubernetes.
+	 *
+	 * @param leaderEventHandler handler for the leader election driver to process leader events.
+	 * @param leaderContenderDescription leader contender description.

Review comment:
       `@throws` is missing.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionEventHandler.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.leaderelection;
+
+/**
+ * Interface which should be implemented to response to {@link LeaderInformation} changes in

Review comment:
       typo: response -> respond

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
##########
@@ -109,9 +109,28 @@
 			final int retries,
 			final Executor executor) {
 
+		return retry(operation, retries, ignore -> true, executor);
+	}
+
+	/**
+	 * Retry the given operation the given number of times in case of a failure only when an exception is retryable.
+	 *
+	 * @param operation to executed
+	 * @param retries if the operation failed
+	 * @param retryPredicate Predicate to test whether an exception is retryable
+	 * @param executor to use to run the futures
+	 * @param <T> type of the result
+	 * @return Future containing either the result of the operation or a {@link RetryException}
+	 */
+	public static <T> CompletableFuture<T> retry(
+		final Supplier<CompletableFuture<T>> operation,
+		final int retries,
+		final Predicate<Throwable> retryPredicate,
+		final Executor executor) {
+

Review comment:
       This is a very nice solution to the problem @wangyang0918. Well done!

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve the leader information from
+ * different storage. The leader address as well as the current leader session ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService, LeaderRetrievalEventHandler {
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderRetrievalDriverFactory leaderRetrievalDriverFactory;
+
+	@GuardedBy("lock")
+	private String lastLeaderAddress;
+
+	@GuardedBy("lock")
+	private UUID lastLeaderSessionID;

Review comment:
       `@Nullable` is missing

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalEventHandler.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+
+/**
+ * Interface which should be implemented to notify to {@link LeaderInformation} changes in
+ * {@link LeaderRetrievalDriver}.
+ */
+public interface LeaderRetrievalEventHandler {
+
+	/**
+	 * Called by specific {@link LeaderRetrievalDriver} to notify leader address.

Review comment:
       line break missing to separate `@param`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalDriver.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link ZooKeeperLeaderElectionDriver}.
+ * {@link LeaderRetrievalService} implementation for Zookeeper. It retrieves the current leader which has
+ * been elected by the {@link ZooKeeperLeaderElectionDriver}.
+ * The leader address as well as the current leader session ID is retrieved from ZooKeeper.
+ */
+public class ZooKeeperLeaderRetrievalDriver implements LeaderRetrievalDriver, NodeCacheListener, UnhandledErrorListener {
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderRetrievalDriver.class);
+
+	/** Connection to the used ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe to watch changes of a specific ZooKeeper node. */
+	private final NodeCache cache;
+
+	private final String retrievalPath;
+
+	private final ConnectionStateListener connectionStateListener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderRetrievalEventHandler leaderRetrievalEventHandler;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a leader retrieval service which uses ZooKeeper to retrieve the leader information.
+	 *
+	 * @param client Client which constitutes the connection to the ZooKeeper quorum
+	 * @param retrievalPath Path of the ZooKeeper node which contains the leader information
+	 * @param leaderRetrievalEventHandler handler to notify the leader changes.
+	 */
+	public ZooKeeperLeaderRetrievalDriver(
+			CuratorFramework client,
+			String retrievalPath,
+			LeaderRetrievalEventHandler leaderRetrievalEventHandler) throws Exception {
+		this.client = checkNotNull(client, "CuratorFramework client");
+		this.cache = new NodeCache(client, retrievalPath);
+		this.retrievalPath = checkNotNull(retrievalPath);
+
+		this.leaderRetrievalEventHandler = checkNotNull(leaderRetrievalEventHandler);
+
+		client.getUnhandledErrorListenable().addListener(this);
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(connectionStateListener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception {
+		if (!running) {
+			return;
+		}
+
+		running = false;

Review comment:
       I'd suggest to introduce a `lock` to guard access to running (also at other places where we access this field). That way we will make sure that after `close` is called that we won't see any calls coming from the `ZooKeeper` threads.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalEventHandler.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+
+/**
+ * Interface which should be implemented to notify to {@link LeaderInformation} changes in
+ * {@link LeaderRetrievalDriver}.
+ */
+public interface LeaderRetrievalEventHandler {
+
+	/**
+	 * Called by specific {@link LeaderRetrievalDriver} to notify leader address.
+	 * @param leaderInformation the new leader information to notify {@link DefaultLeaderRetrievalService}. It could be
+	 * {@link LeaderInformation#empty()} if the leader address does not exist in the external storage.
+	 */
+	void notifyLeaderAddress(LeaderInformation leaderInformation);
+
+	/**
+	 * Handle error by specific {@link LeaderRetrievalDriver}.

Review comment:
       line break is missing

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderRetrievalListener} and
+ * {@link org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler}.
+ */
+public class TestingRetrievalBase {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+	private final Object lock = new Object();
+
+	private String address;
+	private String oldAddress;
+	private UUID leaderSessionID;
+	private Exception exception;
+
+
+	public String getAddress() {
+		return address;
+	}
+
+	public UUID getLeaderSessionID() {
+		return leaderSessionID;
+	}
+
+	public String waitForNewLeader(long timeout) throws Exception {
+		long start = System.currentTimeMillis();
+		long curTimeout;
+
+		synchronized (lock) {
+			while (
+				exception == null &&
+					(address == null || address.equals(oldAddress)) &&
+					(curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
+				try {
+					lock.wait(curTimeout);
+				} catch (InterruptedException e) {
+					// we got interrupted so check again for the condition
+				}
+			}
+		}

Review comment:
       I am wondering whether we can't solve this problem a bit more elegantly using a blocking array queue.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionEventHandler.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.leaderelection;
+
+/**
+ * Interface which should be implemented to response to {@link LeaderInformation} changes in
+ * {@link LeaderElectionDriver}.
+ */
+public interface LeaderElectionEventHandler {
+
+	/**
+	 * Called by specific {@link LeaderElectionDriver} when the leadership is granted.
+	 */
+	void onGrantLeadership();
+
+	/**
+	 * Called by specific {@link LeaderElectionDriver} when the leadership is revoked.
+	 */
+	void onRevokeLeadership();
+
+	/**
+	 * Called by specific {@link LeaderElectionDriver} when the leader information is changed. Then the
+	 * {@link DefaultLeaderElectionService} could write the leader information again if necessary.
+	 * @param leaderInformation leader information which contains leader session id and leader address.

Review comment:
       Will this method only called if one is still the leader or will the implementor be responsible for ensuring that we are still the leader? I think this kind of contracts need to be stated so that future implementations can respect it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionEventHandler.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.leaderelection;
+
+/**
+ * Interface which should be implemented to response to {@link LeaderInformation} changes in
+ * {@link LeaderElectionDriver}.
+ */
+public interface LeaderElectionEventHandler {
+
+	/**
+	 * Called by specific {@link LeaderElectionDriver} when the leadership is granted.
+	 */
+	void onGrantLeadership();
+
+	/**
+	 * Called by specific {@link LeaderElectionDriver} when the leadership is revoked.
+	 */
+	void onRevokeLeadership();
+
+	/**
+	 * Called by specific {@link LeaderElectionDriver} when the leader information is changed. Then the
+	 * {@link DefaultLeaderElectionService} could write the leader information again if necessary.
+	 * @param leaderInformation leader information which contains leader session id and leader address.
+	 */
+	void onLeaderInformationChange(LeaderInformation leaderInformation);
+
+	/**
+	 * Handle error by specific {@link LeaderElectionDriver}.
+	 * @param ex exception to be handled.
+	 */
+	void handleError(Exception ex);

Review comment:
       Theoretically this does not need to be part of the `LeaderElectionEventHandler` and one could instead pass in a `FatalErrorHandler` to the `LeaderElectionDriver`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderInformation.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.runtime.leaderelection;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * Information about leader including the confirmed leader session id and leader address.
+ */
+public class LeaderInformation implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	@Nullable
+	private final UUID leaderSessionID;
+
+	@Nullable
+	private final String leaderAddress;
+
+	private static final LeaderInformation EMPTY = new LeaderInformation(null, null);
+
+	public LeaderInformation(@Nullable UUID leaderSessionID, @Nullable String leaderAddress) {

Review comment:
       Let's make this private and introduce two factory methods `known(String, UUID)` and `empty()`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalDriver.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.leaderretrieval;
+
+/**
+ * A {@link LeaderRetrievalDriver} is responsible for retrieves the current leader which has been elected by the
+ * {@link org.apache.flink.runtime.leaderelection.LeaderElectionDriver}.
+ */
+public interface LeaderRetrievalDriver extends AutoCloseable {
+
+	/**
+	 * Close the services used for leader retrieval.
+	 */
+	void close() throws Exception;

Review comment:
       Shouldn't this also be defined in `AutoCloseable`? Maybe `LeaderRetrievalDriver` does not need to extend from `AutoCloseable`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve the leader information from
+ * different storage. The leader address as well as the current leader session ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService, LeaderRetrievalEventHandler {
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderRetrievalDriverFactory leaderRetrievalDriverFactory;
+
+	@GuardedBy("lock")
+	private String lastLeaderAddress;

Review comment:
       `@Nullable` missing

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionEventHandler.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.leaderelection;
+
+/**
+ * Interface which should be implemented to response to {@link LeaderInformation} changes in
+ * {@link LeaderElectionDriver}.

Review comment:
       Maybe one could say that it reacts also to leader changes and not only to leader information changes.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.leaderelection;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest {

Review comment:
       `extends TestLogger` is missing

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,292 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final String leaderContenderDescription;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler event handler for processing leader change events
+	 * @param leaderContenderDescription leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		if (!running) {
+			return;
+		}
+		running = false;
+
+		LOG.info("Closing {}", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+
+		client.getConnectionStateListenable().removeListener(listener);
+
+		Exception exception = null;
+
+		try {
+			cache.close();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		try {
+			leaderLatch.close();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionDriver.", exception);
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		return leaderLatch.hasLeadership();
+	}
+
+	@Override
+	public void isLeader() {
+		leaderElectionEventHandler.onGrantLeadership();
+	}
+
+	@Override
+	public void notLeader() {
+		leaderElectionEventHandler.onRevokeLeadership();
+	}
+
+	@Override
+	public void nodeChanged() throws Exception {
+		if (leaderLatch.hasLeadership()) {
+			if (running) {
+				ChildData childData = cache.getCurrentData();
+				if (childData != null) {
+					final byte[] data = childData.getData();
+					if (data != null && data.length > 0) {
+						final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+						final ObjectInputStream ois = new ObjectInputStream(bais);
+
+						final String leaderAddress = ois.readUTF();
+						final UUID leaderSessionID = (UUID) ois.readObject();
+
+						leaderElectionEventHandler.onLeaderInformationChange(
+							new LeaderInformation(leaderSessionID, leaderAddress));
+						return;
+					}
+				}
+				leaderElectionEventHandler.onLeaderInformationChange(LeaderInformation.empty());
+			} else {
+				LOG.debug("Ignoring node change notification since the service has already been stopped.");
+			}
+		}
+	}
+
+	/**
+	 * Writes the current leader's address as well the given leader session ID to ZooKeeper.
+	 */
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		// this method does not have to be synchronized because the curator framework client
+		// is thread-safe
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(
+					"Write leader information: Leader={}, session ID={}.",
+					confirmedLeaderAddress,
+					confirmedLeaderSessionID);
+			}
+			final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+			final ObjectOutputStream oos = new ObjectOutputStream(baos);
+
+			oos.writeUTF(confirmedLeaderAddress);
+			oos.writeObject(confirmedLeaderSessionID);
+
+			oos.close();
+
+			boolean dataWritten = false;
+
+			while (!dataWritten && leaderLatch.hasLeadership()) {
+				Stat stat = client.checkExists().forPath(leaderPath);
+
+				if (stat != null) {
+					long owner = stat.getEphemeralOwner();
+					long sessionID = client.getZookeeperClient().getZooKeeper().getSessionId();
+
+					if (owner == sessionID) {
+						try {
+							client.setData().forPath(leaderPath, baos.toByteArray());
+
+							dataWritten = true;
+						} catch (KeeperException.NoNodeException noNode) {
+							// node was deleted in the meantime
+						}
+					} else {
+						try {
+							client.delete().forPath(leaderPath);
+						} catch (KeeperException.NoNodeException noNode) {
+							// node was deleted in the meantime --> try again
+						}
+					}
+				} else {
+					try {
+						client.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(
+								leaderPath,
+								baos.toByteArray());
+
+						dataWritten = true;
+					} catch (KeeperException.NodeExistsException nodeExists) {
+						// node has been created in the meantime --> try again
+					}
+				}
+			}
+
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(
+					"Successfully wrote leader information: Leader={}, session ID={}.",
+					confirmedLeaderAddress,
+					confirmedLeaderSessionID);
+			}
+		} catch (Exception e) {
+			leaderElectionEventHandler.handleError(
+					new Exception("Could not write leader address and leader session ID to " +
+							"ZooKeeper.", e));
+		}
+	}
+
+	private void handleStateChange(ConnectionState newState) {
+		switch (newState) {
+			case CONNECTED:
+				LOG.debug("Connected to ZooKeeper quorum. Leader election can start.");
+				break;
+			case SUSPENDED:
+				LOG.warn("Connection to ZooKeeper suspended. The contender " + leaderContenderDescription
+					+ " no longer participates in the leader election.");
+				break;
+			case RECONNECTED:
+				LOG.info("Connection to ZooKeeper was reconnected. Leader election can be restarted.");
+				break;
+			case LOST:
+				// Maybe we have to throw an exception here to terminate the JobManager
+				LOG.warn("Connection to ZooKeeper lost. The contender " + leaderContenderDescription
+					+ " no longer participates in the leader election.");
+				break;
+		}
+	}
+
+	@Override
+	public void unhandledError(String message, Throwable e) {
+		leaderElectionEventHandler.handleError(
+			new FlinkException("Unhandled error in ZooKeeperLeaderElectionDriver: " + message, e));
+	}
+
+	@Override
+	public String toString() {
+		return "ZooKeeperLeaderElectionDriver{" +
+			"leaderPath='" + leaderPath + '\'' +
+			'}';
+	}
+}

Review comment:
       Very nice! I like the implementation and I think that it is now quite clear and easy to understand what is happening :-)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalEventHandler.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+
+/**
+ * Interface which should be implemented to notify to {@link LeaderInformation} changes in
+ * {@link LeaderRetrievalDriver}.
+ */
+public interface LeaderRetrievalEventHandler {
+
+	/**
+	 * Called by specific {@link LeaderRetrievalDriver} to notify leader address.
+	 * @param leaderInformation the new leader information to notify {@link DefaultLeaderRetrievalService}. It could be

Review comment:
       Usually one does not refers to implementations in the interface definition because the interface does not know about them.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.leaderelection;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		final TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory testingLeaderElectionDriverFactory =
+			new TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory();
+		final DefaultLeaderElectionService leaderElectionService = new DefaultLeaderElectionService(
+			testingLeaderElectionDriverFactory);
+		final TestingContender testingContender = new TestingContender(TEST_URL, leaderElectionService);
+		leaderElectionService.start(testingContender);
+
+		// grant leadership
+		final TestingLeaderElectionDriver testingLeaderElectionDriver =
+			testingLeaderElectionDriverFactory.getCurrentLeaderDriver();
+		assertThat(testingLeaderElectionDriver, is(notNullValue()));
+		testingLeaderElectionDriver.isLeader();
+
+		testingContender.waitForLeader(timeout);
+		assertThat(testingContender.isLeader(), is(true));
+		assertThat(testingContender.getDescription(), is(TEST_URL));
+
+		// Check the external storage
+		assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+
+		// revoke leadership
+		testingLeaderElectionDriver.notLeader();
+		testingContender.waitForRevokeLeader(timeout);
+		assertThat(testingContender.isLeader(), is(false));
+
+		leaderElectionService.stop();
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		final TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory testingLeaderElectionDriverFactory =
+			new TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory();
+		final DefaultLeaderElectionService leaderElectionService = new DefaultLeaderElectionService(
+			testingLeaderElectionDriverFactory);
+		final TestingContender testingContender = new TestingContender(TEST_URL, leaderElectionService);
+		leaderElectionService.start(testingContender);
+
+		final TestingLeaderElectionDriver testingLeaderElectionDriver =
+			testingLeaderElectionDriverFactory.getCurrentLeaderDriver();
+		assertThat(testingLeaderElectionDriver, is(notNullValue()));
+		testingLeaderElectionDriver.isLeader();
+		testingContender.waitForLeader(timeout);
+
+		// Leader information changed and should be corrected
+		testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+		assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));

Review comment:
       Shouldn't we also check that the leader session id stays the same?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalDriver.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link ZooKeeperLeaderElectionDriver}.
+ * {@link LeaderRetrievalService} implementation for Zookeeper. It retrieves the current leader which has
+ * been elected by the {@link ZooKeeperLeaderElectionDriver}.
+ * The leader address as well as the current leader session ID is retrieved from ZooKeeper.
+ */
+public class ZooKeeperLeaderRetrievalDriver implements LeaderRetrievalDriver, NodeCacheListener, UnhandledErrorListener {
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderRetrievalDriver.class);
+
+	/** Connection to the used ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe to watch changes of a specific ZooKeeper node. */
+	private final NodeCache cache;
+
+	private final String retrievalPath;
+
+	private final ConnectionStateListener connectionStateListener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderRetrievalEventHandler leaderRetrievalEventHandler;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a leader retrieval service which uses ZooKeeper to retrieve the leader information.
+	 *
+	 * @param client Client which constitutes the connection to the ZooKeeper quorum
+	 * @param retrievalPath Path of the ZooKeeper node which contains the leader information
+	 * @param leaderRetrievalEventHandler handler to notify the leader changes.
+	 */
+	public ZooKeeperLeaderRetrievalDriver(
+			CuratorFramework client,
+			String retrievalPath,
+			LeaderRetrievalEventHandler leaderRetrievalEventHandler) throws Exception {
+		this.client = checkNotNull(client, "CuratorFramework client");
+		this.cache = new NodeCache(client, retrievalPath);
+		this.retrievalPath = checkNotNull(retrievalPath);
+
+		this.leaderRetrievalEventHandler = checkNotNull(leaderRetrievalEventHandler);
+
+		client.getUnhandledErrorListenable().addListener(this);
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(connectionStateListener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception {
+		if (!running) {
+			return;
+		}
+
+		running = false;
+
+		LOG.info("Closing {}.", this);
+
+		client.getUnhandledErrorListenable().removeListener(this);
+		client.getConnectionStateListenable().removeListener(connectionStateListener);
+
+		try {
+			cache.close();
+		} catch (IOException e) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderRetrievalDriver.", e);
+		}
+	}
+
+	@Override
+	public void nodeChanged() {
+		retrieveLeaderInformationFromZooKeeper();
+	}
+
+	private void retrieveLeaderInformationFromZooKeeper() {
+		if (running) {
+			try {
+				LOG.debug("Leader node has changed.");
+
+				ChildData childData = cache.getCurrentData();
+
+				String leaderAddress;
+				UUID leaderSessionID;
+
+				if (childData == null) {
+					leaderAddress = null;
+					leaderSessionID = null;
+				} else {
+					byte[] data = childData.getData();
+
+					if (data == null || data.length == 0) {
+						leaderAddress = null;
+						leaderSessionID = null;
+					} else {
+						ByteArrayInputStream bais = new ByteArrayInputStream(data);
+						ObjectInputStream ois = new ObjectInputStream(bais);
+
+						leaderAddress = ois.readUTF();
+						leaderSessionID = (UUID) ois.readObject();
+					}
+				}
+
+				leaderRetrievalEventHandler.notifyLeaderAddress(new LeaderInformation(leaderSessionID, leaderAddress));
+			} catch (Exception e) {
+				leaderRetrievalEventHandler.handleError(new Exception("Could not handle node changed event.", e));
+				ExceptionUtils.checkInterrupted(e);
+			}
+		} else {
+			LOG.debug("Ignoring node change notification since the service has already been stopped.");
+		}
+	}
+
+	private void handleStateChange(ConnectionState newState) {
+		switch (newState) {
+			case CONNECTED:
+				LOG.debug("Connected to ZooKeeper quorum. Leader retrieval can start.");
+				break;
+			case SUSPENDED:
+				LOG.warn("Connection to ZooKeeper suspended. Can no longer retrieve the leader from " +
+					"ZooKeeper.");
+				leaderRetrievalEventHandler.notifyLeaderAddress(LeaderInformation.empty());

Review comment:
       I think we should check whether we are still running before calling this method.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.leaderelection;
+
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link DefaultLeaderElectionService}.
+ */
+public class DefaultLeaderElectionServiceTest {
+
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final long timeout = 30L * 1000L;
+
+	@Test
+	public void testOnGrantAndRevokeLeadership() throws Exception {
+		final TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory testingLeaderElectionDriverFactory =
+			new TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory();
+		final DefaultLeaderElectionService leaderElectionService = new DefaultLeaderElectionService(
+			testingLeaderElectionDriverFactory);
+		final TestingContender testingContender = new TestingContender(TEST_URL, leaderElectionService);
+		leaderElectionService.start(testingContender);
+
+		// grant leadership
+		final TestingLeaderElectionDriver testingLeaderElectionDriver =
+			testingLeaderElectionDriverFactory.getCurrentLeaderDriver();
+		assertThat(testingLeaderElectionDriver, is(notNullValue()));
+		testingLeaderElectionDriver.isLeader();
+
+		testingContender.waitForLeader(timeout);
+		assertThat(testingContender.isLeader(), is(true));
+		assertThat(testingContender.getDescription(), is(TEST_URL));
+
+		// Check the external storage
+		assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+
+		// revoke leadership
+		testingLeaderElectionDriver.notLeader();
+		testingContender.waitForRevokeLeader(timeout);
+		assertThat(testingContender.isLeader(), is(false));
+
+		leaderElectionService.stop();
+	}
+
+	@Test
+	public void testLeaderInformationChangedAndShouldBeCorrected() throws Exception {
+		final TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory testingLeaderElectionDriverFactory =
+			new TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory();
+		final DefaultLeaderElectionService leaderElectionService = new DefaultLeaderElectionService(
+			testingLeaderElectionDriverFactory);
+		final TestingContender testingContender = new TestingContender(TEST_URL, leaderElectionService);
+		leaderElectionService.start(testingContender);
+
+		final TestingLeaderElectionDriver testingLeaderElectionDriver =
+			testingLeaderElectionDriverFactory.getCurrentLeaderDriver();
+		assertThat(testingLeaderElectionDriver, is(notNullValue()));
+		testingLeaderElectionDriver.isLeader();
+		testingContender.waitForLeader(timeout);
+
+		// Leader information changed and should be corrected
+		testingLeaderElectionDriver.leaderInformationChanged(LeaderInformation.empty());
+		assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));
+
+		testingLeaderElectionDriver.leaderInformationChanged(
+			new LeaderInformation(UUID.randomUUID(), "faulty-address"));
+		assertThat(testingLeaderElectionDriver.getLeaderInformation().getLeaderAddress(), is(TEST_URL));

Review comment:
       Same here with the leader session id.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.leaderelection;
+
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderContender} and
+ * {@link LeaderElectionEventHandler}.
+ */
+public class TestingLeaderBase {
+
+	protected boolean leader = false;
+	protected Throwable error = null;
+
+	protected final Object lock = new Object();
+	private final Object errorLock = new Object();
+
+	/**
+	 * Waits until the contender becomes the leader or until the timeout has been exceeded.
+	 *
+	 * @param timeout
+	 * @throws TimeoutException
+	 */
+	public void waitForLeader(long timeout) throws TimeoutException {
+		waitFor(this::isLeader, timeout, "Contender was not elected as the leader within " + timeout + "ms");
+	}
+
+	/**
+	 * Waits until the contender revokes the leader or until the timeout has been exceeded.
+	 *
+	 * @param timeout
+	 * @throws TimeoutException
+	 */
+	public void waitForRevokeLeader(long timeout) throws TimeoutException {
+		waitFor(() -> !isLeader(), timeout, "Contender was not revoked within " + timeout + "ms");
+	}
+
+	protected void waitFor(Supplier<Boolean> supplier, long timeout, String msg) throws TimeoutException {
+		long start = System.currentTimeMillis();
+		long curTimeout;
+
+		while (!supplier.get() && (curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
+			synchronized (lock) {
+				try {
+					lock.wait(curTimeout);
+				} catch (InterruptedException e) {
+					// we got interrupted so check again for the condition
+				}
+			}
+		}
+
+		if (!supplier.get()) {
+			throw new TimeoutException(msg);
+		}

Review comment:
       I am wondering whether we can't solve this problem a bit more elegantly using a blocking array queue.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,292 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+	/** Client to the ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election. */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes. */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information. */
+	private final String leaderPath;
+
+	private final ConnectionStateListener listener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final String leaderContenderDescription;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionDriver object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 * @param leaderElectionEventHandler event handler for processing leader change events
+	 * @param leaderContenderDescription leader contender description
+	 */
+	public ZooKeeperLeaderElectionDriver(
+			CuratorFramework client,
+			String latchPath,
+			String leaderPath,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			String leaderContenderDescription) throws Exception {
+		this.client = checkNotNull(client);
+		this.leaderPath = checkNotNull(leaderPath);
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler);
+		this.leaderContenderDescription = checkNotNull(leaderContenderDescription);
+
+		leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+		cache = new NodeCache(client, leaderPath);
+
+		client.getUnhandledErrorListenable().addListener(this);
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(listener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception{
+		if (!running) {
+			return;
+		}
+		running = false;

Review comment:
       I'd suggest to introduce a `lock` to guard accesses to `running` and all other places where it is accessed. That way we will prevent concurrency when shutting this component down and when the `ZooKeeper` threads trigger an update.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r506243026



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,68 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		return CompletableFuture.runAsync(
+			() -> {
+				if (!getConfigMap(configMap.getName()).isPresent()) {
+					this.internalClient.configMaps().create(configMap.getInternalResource());
+				}

Review comment:
       If the ConfigMap already exists, the create operation will fail. Since the check-and-create is not an atomic operation, I will use a `try... catch` to directly creating the ConfigMap.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 1e31aa2c0c7217ba82db789933651036d2fa7e2d Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385) 
   * 37efc6f14c55e2bd261b70924c9525c6539447f8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508401521



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.UUID;
+
+/**
+ * Abstract class for leader election service based on distributed coordination system(e.g. Zookeeper, Kubernetes, etc.).
+ */
+public abstract class AbstractLeaderElectionService implements LeaderElectionService {
+
+	protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+	protected final Object lock = new Object();
+
+	/** The leader contender which applies for leadership. */
+	protected volatile LeaderContender leaderContender;
+
+	private volatile UUID issuedLeaderSessionID;
+
+	protected volatile UUID confirmedLeaderSessionID;
+
+	protected volatile String confirmedLeaderAddress;
+
+	protected volatile boolean running;
+
+	protected AbstractLeaderElectionService() {
+		leaderContender = null;
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		confirmedLeaderAddress = null;
+
+		running = false;
+	}
+
+	@Override
+	public final void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		logger.info("Starting LeaderElectionService {}.", this);
+
+		synchronized (lock) {
+			leaderContender = contender;
+			running = true;
+			internalStart(contender);
+		}
+	}
+
+	@Override
+	public final void stop() throws Exception {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+			clearConfirmedLeaderInformation();
+		}
+
+		logger.info("Stopping LeaderElectionService {}.", this);
+
+		internalStop();
+	}
+
+	@Override
+	public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {
+		if (logger.isDebugEnabled()) {
+			logger.debug(
+				"Confirm leader session ID {} for leader {}.",
+				leaderSessionID,
+				leaderAddress);
+		}
+
+		Preconditions.checkNotNull(leaderSessionID);
+
+		if (checkLeaderLatch()) {

Review comment:
       I think we can keep `checkLeaderLatch` in `ZooKeeperLeaderElectionService` , while calling it from the ZK implementation of `hasLeadership`. I don't see a good reason for keeping a non-argument `hasLeaderShip` in `AbstractLeaderElectionService`. How should the abstract class define the semantics and contract of the non-argument `hasLeaderShip` when it is expected not aware of the leader latch (a ZK implementation detail)?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * f6be1c1045c89a0c2a2d8f1d317d41a656cae538 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895) 
   * 5e2cfa21731b9fb5dd976cce821bfda78daa91b7 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507604234



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesHighAvailabilityOptions;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.NAME_SEPARATOR;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * High availability service for Kubernetes.
+ */
+public class KubernetesHaServices implements HighAvailabilityServices {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesHaServices.class);
+
+	private static final String RESOURCE_MANAGER_NAME = "resourcemanager";
+
+	private static final String DISPATCHER_NAME = "dispatcher";
+
+	private static final String JOB_MANAGER_NAME = "jobmanager";
+
+	private static final String REST_SERVER_NAME = "restserver";
+
+	private final String leaderSuffix;
+
+	private final String clusterId;
+
+	/** Kubernetes client. */
+	private final FlinkKubeClient kubeClient;
+
+	/** The executor to run Kubernetes operations on. */
+	private final Executor executor;
+
+	/** The runtime configuration. */
+	private final Configuration configuration;
+
+	/** Store for arbitrary blobs. */
+	private final BlobStoreService blobStoreService;
+
+	/** The Kubernetes based running jobs registry. */
+	private final RunningJobsRegistry runningJobsRegistry;

Review comment:
       I will try to abstract these two HA service.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509048050



##########
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:
       Hmm, I think it would be good to actually test the ha services against some form of K8s testing server/cluster. I mean they can run as part of the E2E tests but we should not only test a full Flink cluster with K8s ha enabled.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514067718



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.After;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Base class for high availability unit tests with a configured testing Kubernetes client.
+ */
+public class KubernetesHighAvailabilityTestBase extends TestLogger {
+	private static final String CLUSTER_ID = "leader-test-cluster";
+
+	public static final String LOCK_IDENTITY = UUID.randomUUID().toString();
+	public static final String LEADER_URL = "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher";
+	public static final String LEADER_CONFIGMAP_NAME = "leader-test-cluster";
+
+	protected static final long TIMEOUT = 30L * 1000L;
+
+	protected ExecutorService executorService;
+	protected Configuration configuration;
+
+	@Before
+	public void setup() {
+		configuration = new Configuration();
+		configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
+		executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("IO-Executor"));
+	}
+
+	@After
+	public void teardown() throws Exception {
+		executorService.shutdownNow();
+		executorService.awaitTermination(TIMEOUT, TimeUnit.MILLISECONDS);
+	}
+
+	/**
+	 * Context to leader election and retrieval tests.
+	 */
+	protected class Context {
+		/**
+		 * The configMapStore should only be used for set the following functions. Use a ConcurrentHashMap here since it
+		 * could be updated by {@link TestingFlinkKubeClient.MockKubernetesLeaderElector} and
+		 * {@link KubernetesLeaderElectionDriver}.
+		 */
+		private final Map<String, KubernetesConfigMap> configMapStore = new ConcurrentHashMap<>();
+
+		final AtomicBoolean leaderController = new AtomicBoolean(false);
+
+		final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+			configMapCallbackFutures = new ArrayList<>();
+
+		final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture = new CompletableFuture<>();
+		final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+		final FlinkKubeClient flinkKubeClient;
+
+		final LeaderElectionDriver leaderElectionDriver;
+		final TestingLeaderElectionEventHandler electionEventHandler;
+
+		final LeaderRetrievalDriver leaderRetrievalDriver;
+		final TestingLeaderRetrievalEventHandler retrievalEventHandler;
+
+		Context() {
+			flinkKubeClient = getFlinkKubeClient();
+
+			electionEventHandler = new TestingLeaderElectionEventHandler(LEADER_URL);
+			leaderElectionDriver = createLeaderElectionDriver();
+
+			retrievalEventHandler = new TestingLeaderRetrievalEventHandler();
+			leaderRetrievalDriver = createLeaderRetrievalDriver();
+		}
+
+		void runTestAndGrantLeadershipToContender(RunnableWithException testMethod) throws Exception {
+			// Grant leadership
+			leaderController.set(true);
+			electionEventHandler.waitForLeader(TIMEOUT);
+			assertThat(electionEventHandler.isLeader(), is(true));
+
+			testMethod.run();
+
+			leaderElectionDriver.close();
+			leaderRetrievalDriver.close();
+		}
+
+		KubernetesConfigMap getLeaderConfigMap() {
+			final Optional<KubernetesConfigMap> configMapOpt = flinkKubeClient.getConfigMap(LEADER_CONFIGMAP_NAME);
+			assertThat(configMapOpt.isPresent(), is(true));
+			return configMapOpt.get();
+		}
+
+		protected FlinkKubeClient getFlinkKubeClient() {
+			return TestingFlinkKubeClient.builder()
+				.setCreateConfigMapFunction(configMap -> {
+					configMapStore.put(configMap.getName(), configMap);
+					return CompletableFuture.completedFuture(null);
+				})
+				.setGetConfigMapFunction(configMapName -> Optional.ofNullable(configMapStore.get(configMapName)))
+				.setCheckAndUpdateConfigMapFunction((configMapName, updateFunction) -> {
+					final KubernetesConfigMap configMap = configMapStore.get(configMapName);
+					if (configMap != null) {
+						try {
+							final boolean updated = updateFunction.apply(configMap).map(
+								updateConfigMap -> {
+									configMapStore.put(configMap.getName(), updateConfigMap);
+									return true;
+								}).orElse(false);
+							return CompletableFuture.completedFuture(updated);
+						} catch (Throwable throwable) {
+							throw new CompletionException(new KubernetesException("Failed to execute update fuction"));
+						}
+					}
+					throw new CompletionException(
+						new KubernetesException("ConfigMap " + configMapName + " does not exists."));
+				})
+				.setWatchConfigMapsFunction((ignore, handler) -> {
+					final CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>> future =
+						new CompletableFuture<>();
+					future.complete(handler);
+					configMapCallbackFutures.add(future);
+					return new TestingFlinkKubeClient.MockKubernetesWatch();
+				})
+				.setDeleteConfigMapFunction(name -> {
+					configMapStore.remove(name);
+					return FutureUtils.completedVoidFuture();
+				})
+				.setDeleteConfigMapByLabelFunction(labels -> {
+					deleteConfigMapByLabelsFuture.complete(labels);

Review comment:
       It could only be called once now. I will add a check here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513354997



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalDriver.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link ZooKeeperLeaderElectionDriver}.
+ * {@link LeaderRetrievalService} implementation for Zookeeper. It retrieves the current leader which has
+ * been elected by the {@link ZooKeeperLeaderElectionDriver}.
+ * The leader address as well as the current leader session ID is retrieved from ZooKeeper.
+ */
+public class ZooKeeperLeaderRetrievalDriver implements LeaderRetrievalDriver, NodeCacheListener, UnhandledErrorListener {
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderRetrievalDriver.class);
+
+	/** Connection to the used ZooKeeper quorum. */
+	private final CuratorFramework client;
+
+	/** Curator recipe to watch changes of a specific ZooKeeper node. */
+	private final NodeCache cache;
+
+	private final String retrievalPath;
+
+	private final ConnectionStateListener connectionStateListener = (client, newState) -> handleStateChange(newState);
+
+	private final LeaderRetrievalEventHandler leaderRetrievalEventHandler;
+
+	private volatile boolean running;
+
+	/**
+	 * Creates a leader retrieval service which uses ZooKeeper to retrieve the leader information.
+	 *
+	 * @param client Client which constitutes the connection to the ZooKeeper quorum
+	 * @param retrievalPath Path of the ZooKeeper node which contains the leader information
+	 * @param leaderRetrievalEventHandler handler to notify the leader changes.
+	 */
+	public ZooKeeperLeaderRetrievalDriver(
+			CuratorFramework client,
+			String retrievalPath,
+			LeaderRetrievalEventHandler leaderRetrievalEventHandler) throws Exception {
+		this.client = checkNotNull(client, "CuratorFramework client");
+		this.cache = new NodeCache(client, retrievalPath);
+		this.retrievalPath = checkNotNull(retrievalPath);
+
+		this.leaderRetrievalEventHandler = checkNotNull(leaderRetrievalEventHandler);
+
+		client.getUnhandledErrorListenable().addListener(this);
+		cache.getListenable().addListener(this);
+		cache.start();
+
+		client.getConnectionStateListenable().addListener(connectionStateListener);
+
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception {
+		if (!running) {
+			return;
+		}
+
+		running = false;

Review comment:
       I think we just need to make sure that we don't introduce a deadlock by requesting the inner and outer lock in a problematic order. In order to ensure this we should not call the event handler under the inner lock.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r507371480



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -219,6 +230,68 @@ public KubernetesWatch watchPodsAndDoCallback(
 				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
 	}
 
+	@Override
+	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
+		return CompletableFuture.runAsync(
+			() -> {
+				if (!getConfigMap(configMap.getName()).isPresent()) {
+					this.internalClient.configMaps().create(configMap.getInternalResource());
+				}
+			},
+			kubeClientExecutorService);

Review comment:
       Yes. This is already in the plan. I have left a comments on the FLINK-19037 and created a new ticket FLINK-19700 to follow this.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509276361



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java
##########
@@ -249,4 +265,69 @@ public void testStopAndCleanupCluster() throws Exception {
 		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
 		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
 	}
+
+	@Test
+	public void testCreateConfigMap() throws Exception {
+		final KubernetesConfigMap configMap = buildTestingConfigMap();
+		this.flinkKubeClient.createConfigMap(configMap).get();
+		final Optional<KubernetesConfigMap> currentOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(currentOpt.isPresent(), is(true));
+		assertThat(currentOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	@Test
+	public void testDeleteConfigMapByLabels() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMapsByLabels(TESTING_LABELS);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testDeleteConfigMapByName() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap()).get();
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(true));
+		this.flinkKubeClient.deleteConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME).isPresent(), is(false));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMap() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker pass
+		this.flinkKubeClient.checkAndUpdateConfigMap(
+			TESTING_CONFIG_MAP_NAME,
+			c -> {
+				c.getData().put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_NEW_VALUE);
+				return Optional.of(c);
+			}).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_NEW_VALUE));
+	}
+
+	@Test
+	public void testCheckAndUpdateConfigMapWithEmptyResult() throws Exception {
+		this.flinkKubeClient.createConfigMap(buildTestingConfigMap());
+
+		// Checker not pass and return empty result
+		this.flinkKubeClient.checkAndUpdateConfigMap(TESTING_CONFIG_MAP_NAME, c -> Optional.empty()).get();
+
+		final Optional<KubernetesConfigMap> configMapOpt = this.flinkKubeClient.getConfigMap(TESTING_CONFIG_MAP_NAME);
+		assertThat(configMapOpt.isPresent(), is(true));
+		assertThat(configMapOpt.get().getData().get(TESTING_CONFIG_MAP_KEY), is(TESTING_CONFIG_MAP_VALUE));
+	}
+
+	private KubernetesConfigMap buildTestingConfigMap() {
+		final Map<String, String> data = new HashMap<>();
+		data.put(TESTING_CONFIG_MAP_KEY, TESTING_CONFIG_MAP_VALUE);
+		return new KubernetesConfigMap(new ConfigMapBuilder()
+			.withNewMetadata()
+			.withName(TESTING_CONFIG_MAP_NAME)
+			.withLabels(TESTING_LABELS)
+			.endMetadata()
+			.withData(data).build());
+	}

Review comment:
       I will add a test for creating failed and an exception should be thrown.
   
   The reason why I do not test the concurrent writes is that we are using `KubernetesMockServer`. Actually, it could not support such semantic, a ConfigMap with same name already exists, the creating should failed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 5e2cfa21731b9fb5dd976cce821bfda78daa91b7 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186) 
   * 7ae53916e13b43e94621217d7a354a672077e0ab UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8720",
       "triggerID" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753",
       "triggerID" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884",
       "triggerID" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8956",
       "triggerID" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8988",
       "triggerID" : "e2795d9209fa2241cf0909737df9dca17defea4e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * e2795d9209fa2241cf0909737df9dca17defea4e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8988) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8385",
       "triggerID" : "1e31aa2c0c7217ba82db789933651036d2fa7e2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8420",
       "triggerID" : "37efc6f14c55e2bd261b70924c9525c6539447f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8511",
       "triggerID" : "7f34f6f31596832223b1cf2010bed1c47f7502ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8654",
       "triggerID" : "6a2602f39684ca0005350787b1c8051815acecdc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8720",
       "triggerID" : "f43eb8741c56e21ea34565b26f3deadab182de00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8753",
       "triggerID" : "2317c84938b7434c743b13a3f478d301a4c70443",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884",
       "triggerID" : "ed02e962a354185fb37eaedbdfee217d691bd16b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * ed02e962a354185fb37eaedbdfee217d691bd16b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8884) 
   * e78c6eee6bbcd1809b5b4f1ee65ffc0aa9ff1924 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508995324



##########
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);

Review comment:
       This call could happen late when the IO executor is very busy or the network is not very good. So I will add a checker for whether a new leader has been elected via checking the annotation on ConfigMap.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509888875



##########
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);

Review comment:
       Hmm. After checking the implementation in fabric8 Kubernetes client `LeaderElector` implementation, the leader annotation will not be cleared even the current leadership is revoked. So you are right we could still check the leadership here. Only when the annotation is matched to the current `lockIdentity`, which means a next leader has not been elected, we could safely clear the leader information in the ConfigMap.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-708933599


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7653",
       "triggerID" : "e33c52394024e8f1312c096a4b612d53d8848b97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e60cdbfeb10a33d4df62453679cd309fc39ee198",
       "triggerType" : "PUSH"
     }, {
       "hash" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7874",
       "triggerID" : "991bdfa5427479f845abaf91caba0069011ee9e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7895",
       "triggerID" : "f6be1c1045c89a0c2a2d8f1d317d41a656cae538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186",
       "triggerID" : "5e2cfa21731b9fb5dd976cce821bfda78daa91b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285",
       "triggerID" : "7ae53916e13b43e94621217d7a354a672077e0ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293",
       "triggerID" : "aad1ba94672d89de5a4fc814297f0c40e3552db0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e60cdbfeb10a33d4df62453679cd309fc39ee198 UNKNOWN
   * 5e2cfa21731b9fb5dd976cce821bfda78daa91b7 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8186) 
   * 7ae53916e13b43e94621217d7a354a672077e0ab Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8285) 
   * aad1ba94672d89de5a4fc814297f0c40e3552db0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8293) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-721066047


   @tillrohrmann I have added some `fixup`s to address the comments so that you could be focused on the changes. Also you may need to have a look the unresolved conversation.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-715290752


   @tillrohrmann Thanks for the discussion and review. I have addressed all the comments you have left and force pushed. The PR get bigger, but I think it is more clear.
   The biggest changes are listed as following.
   * Use composition instead of inheritance for `LeaderElectionService` and `LeaderRetrievalService`. Now it is easier to add new implementation of the both services.
   * The `HaServices` abstraction is still using inheritance since it is easy enough to add new implementation. And we could not get too much benefit using composition.
   * When ConfigMap is deleted/updated externally, we will throw error instead of handling internally.
   * Add ITCases for the new introduced leader election/retrieval services
   
   Maybe @xintongsong could also have another look.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513219224



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.leaderelection;
+
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+/**
+ * Base class which provides some convenience functions for testing purposes of {@link LeaderContender} and
+ * {@link LeaderElectionEventHandler}.
+ */
+public class TestingLeaderBase {
+
+	protected boolean leader = false;
+	protected Throwable error = null;
+
+	protected final Object lock = new Object();
+	private final Object errorLock = new Object();
+
+	/**
+	 * Waits until the contender becomes the leader or until the timeout has been exceeded.
+	 *
+	 * @param timeout
+	 * @throws TimeoutException
+	 */
+	public void waitForLeader(long timeout) throws TimeoutException {
+		waitFor(this::isLeader, timeout, "Contender was not elected as the leader within " + timeout + "ms");
+	}
+
+	/**
+	 * Waits until the contender revokes the leader or until the timeout has been exceeded.
+	 *
+	 * @param timeout
+	 * @throws TimeoutException
+	 */
+	public void waitForRevokeLeader(long timeout) throws TimeoutException {
+		waitFor(() -> !isLeader(), timeout, "Contender was not revoked within " + timeout + "ms");
+	}
+
+	protected void waitFor(Supplier<Boolean> supplier, long timeout, String msg) throws TimeoutException {
+		long start = System.currentTimeMillis();
+		long curTimeout;
+
+		while (!supplier.get() && (curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
+			synchronized (lock) {
+				try {
+					lock.wait(curTimeout);
+				} catch (InterruptedException e) {
+					// we got interrupted so check again for the condition
+				}
+			}
+		}
+
+		if (!supplier.get()) {
+			throw new TimeoutException(msg);
+		}

Review comment:
       Hmm. Actually, this is just the old codes I moved from `TestingContender`. It is indeed could be optimized. I will do it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516790760



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientITCase.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.kubeclient;
+
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client.
+ */
+public class Fabric8FlinkKubeClientITCase {
+
+	@ClassRule
+	public static KubernetesResource kubernetesResource = new KubernetesResource();
+
+	private static final String TEST_CONFIG_MAP_NAME = "test-config-map";
+
+	private static final long TIMEOUT = 120L * 1000L;
+
+	private static final Map<String, String> data = new HashMap<String, String>() {
+		{
+			put("key1", "0");
+			put("key2", "0");
+			put("key3", "0");
+		}
+	};
+
+	private FlinkKubeClient flinkKubeClient;
+
+	@Before
+	public void setup() throws Exception {
+		flinkKubeClient = kubernetesResource.getFlinkKubeClient();
+		flinkKubeClient.createConfigMap(new KubernetesConfigMap(
+			new ConfigMapBuilder()
+				.withNewMetadata()
+				.withName(TEST_CONFIG_MAP_NAME)
+				.endMetadata()
+				.withData(data)
+				.build())).get();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		flinkKubeClient.deleteConfigMap(TEST_CONFIG_MAP_NAME).get();
+	}
+
+	/**
+	 * {@link org.apache.flink.kubernetes.kubeclient.FlinkKubeClient#checkAndUpdateConfigMap} is a transactional
+	 * operation, we should definitely guarantee that the concurrent modification could work.
+	 */
+	@Test
+	public void testCheckAndUpdateConfigMapConcurrently() throws Exception {
+		// Start multiple instances to update ConfigMap concurrently
+		final List<CompletableFuture<Void>> futures = new ArrayList<>();
+		final int target = 10;
+		final int updateIntervalMs = 100;
+		for (String key : data.keySet()) {
+			futures.add(FutureUtils.runAfterwardsAsync(FutureUtils.completedVoidFuture(), () -> {

Review comment:
       If you use `join` instead of `get`, then you could use `CompletableFuture.runAsync(() -> {}, exec)`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r512440752



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.leaderelection;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could
+ * perform a leader election for the contender, and then persist the leader information to various storage.
+ */
+public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderElectionDriverFactory leaderElectionDriverFactory;
+
+	/** The leader contender which applies for leadership. */
+	@GuardedBy("lock")
+	private volatile LeaderContender leaderContender;
+
+	@GuardedBy("lock")
+	private volatile UUID issuedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile UUID confirmedLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile String confirmedLeaderAddress;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	@GuardedBy("lock")
+	private LeaderElectionDriver leaderElectionDriver;
+
+	public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) {
+		this.leaderElectionDriverFactory = leaderElectionDriverFactory;

Review comment:
       minor: null check

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderInformation.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.leaderelection;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * Information about leader including the confirmed leader session id and leader address.
+ */
+public class LeaderInformation implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final UUID leaderSessionID;
+
+	private final String leaderAddress;
+
+	private static final LeaderInformation EMPTY = new LeaderInformation(null, null);
+
+	public LeaderInformation(@Nullable UUID leaderSessionID, @Nullable String leaderAddress) {
+		this.leaderSessionID = leaderSessionID;
+		this.leaderAddress = leaderAddress;
+	}
+
+	@Nullable
+	public UUID getLeaderSessionID() {
+		return leaderSessionID;
+	}
+
+	@Nullable
+	public String getLeaderAddress() {
+		return leaderAddress;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj == this) {
+			return true;
+		} else if (obj != null && obj.getClass() == LeaderInformation.class) {
+			final LeaderInformation that = (LeaderInformation) obj;
+			return Objects.equals(this.leaderSessionID, that.leaderSessionID) &&
+				Objects.equals(this.leaderAddress, that.leaderAddress);
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public int hashCode() {
+		int result = leaderSessionID.hashCode();

Review comment:
       This may throw NPE if `leaderSessionID` is `null`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderInformation.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.leaderelection;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * Information about leader including the confirmed leader session id and leader address.
+ */
+public class LeaderInformation implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final UUID leaderSessionID;
+
+	private final String leaderAddress;

Review comment:
       These 2 fields should also be annotated with `@Nullable`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve the leader information from
+ * different storage. The leader address as well as the current leader session ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService, LeaderRetrievalEventHandler {
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+	private final Object lock = new Object();
+
+	private final LeaderRetrievalDriverFactory leaderRetrievalDriverFactory;
+
+	@GuardedBy("lock")
+	private String lastLeaderAddress;
+
+	@GuardedBy("lock")
+	private UUID lastLeaderSessionID;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	/** Listener which will be notified about leader changes. */
+	@GuardedBy("lock")
+	private volatile LeaderRetrievalListener leaderListener;
+
+	@GuardedBy("lock")
+	private LeaderRetrievalDriver leaderRetrievalDriver;
+
+	/**
+	 * Creates a default leader retrieval service with specified {@link LeaderRetrievalDriverFactory}.
+	 *
+	 * @param leaderRetrievalDriverFactory {@link LeaderRetrievalDriverFactory} used for creating
+	 * {@link LeaderRetrievalDriver}.
+	 */
+	public DefaultLeaderRetrievalService(LeaderRetrievalDriverFactory leaderRetrievalDriverFactory) {
+		this.leaderRetrievalDriverFactory = leaderRetrievalDriverFactory;

Review comment:
       null check




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r508335109



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -104,6 +107,67 @@ KubernetesWatch watchPodsAndDoCallback(
 		Map<String, String> labels,
 		WatchCallbackHandler<KubernetesPod> podCallbackHandler);
 
+	/**
+	 * Create the ConfigMap with specified content. If the ConfigMap already exists, nothing will happen.
+	 *
+	 * @param configMap ConfigMap.
+	 *
+	 * @return Return the ConfigMap create future.
+	 */
+	CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap);

Review comment:
       Makes sense to me.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r509258549



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Abstract high availability services based on distributed system(e.g. Zookeeper, Kubernetes).
+ */
+public abstract class AbstractHaServices implements HighAvailabilityServices {

Review comment:
       To avoid the renaming issue one could keep `ZooKeeperHaServices` as an alias for the `HaServices` with a `ZooKeeperDriver` (e.g. `ZooKeeperHaServices extend HaServices` and then in the constructor `super(new ZooKeeperDriver())`), for example. But I haven't checked it very thoroughly.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r516788903



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.runtime.leaderelection.LeaderInformation;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_KEY;
+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.hamcrest.Matchers.containsString;
+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 KubernetesLeaderElectionDriver}.
+ */
+public class KubernetesLeaderElectionDriverTest extends KubernetesHighAvailabilityTestBase {
+
+	@Test
+	public void testIsLeader() throws Exception {
+		new Context() {{
+			runTest(
+				() -> {
+					// Grant leadership
+					leaderCallbackGrantLeadership();
+					assertThat(electionEventHandler.isLeader(), is(true));
+					assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LEADER_INFORMATION));
+				});
+		}};
+	}
+
+	@Test
+	public void testNotLeader() throws Exception {
+		new Context() {{
+			runTest(
+				() -> {
+					leaderCallbackGrantLeadership();
+					// Revoke leadership
+					getLeaderCallback().notLeader();
+
+					electionEventHandler.waitForRevokeLeader(TIMEOUT);
+					assertThat(electionEventHandler.isLeader(), is(false));
+					assertThat(electionEventHandler.getConfirmedLeaderInformation(), is(LeaderInformation.empty()));
+					// The ConfigMap should also be cleared
+					assertThat(getLeaderConfigMap().getData().get(LEADER_ADDRESS_KEY), is(nullValue()));
+					assertThat(getLeaderConfigMap().getData().get(LEADER_SESSION_ID_KEY), is(nullValue()));

Review comment:
       ah ok. Thanks for the explanation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r514063391



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriver.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.configuration.KubernetesLeaderElectionConfiguration;
+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.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionDriver;
+import org.apache.flink.runtime.leaderelection.LeaderElectionEventHandler;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+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.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.getLeaderInformationFromConfigMap;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Kubernetes. The active leader 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 KubernetesLeaderElectionDriver implements LeaderElectionDriver {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderElectionDriver.class);
+
+	private final Object lock = new Object();
+
+	private final FlinkKubeClient kubeClient;
+
+	private final Executor ioExecutor;
+
+	private final String configMapName;
+
+	private final String lockIdentity;
+
+	private final KubernetesLeaderElector leaderElector;
+
+	// Labels will be used to clean up the ha related ConfigMaps.
+	private final Map<String, String> configMapLabels;
+
+	private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+	private final KubernetesWatch kubernetesWatch;
+
+	private final FatalErrorHandler fatalErrorHandler;
+
+	@GuardedBy("lock")
+	private volatile CompletableFuture<Void> leaderElectorRunFuture;
+
+	@GuardedBy("lock")
+	private volatile boolean running;
+
+	public KubernetesLeaderElectionDriver(
+			FlinkKubeClient kubeClient,
+			Executor ioExecutor,
+			KubernetesLeaderElectionConfiguration leaderConfig,
+			LeaderElectionEventHandler leaderElectionEventHandler,
+			FatalErrorHandler fatalErrorHandler) {
+
+		this.kubeClient = checkNotNull(kubeClient, "Kubernetes client");
+		this.ioExecutor = checkNotNull(ioExecutor, "IO Executor");
+
+		this.leaderElectionEventHandler = checkNotNull(leaderElectionEventHandler, "LeaderElectionEventHandler");
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.configMapName = leaderConfig.getConfigMapName();
+		this.lockIdentity = leaderConfig.getLockIdentity();
+		this.leaderElector = kubeClient.createLeaderElector(leaderConfig, new LeaderCallbackHandlerImpl());
+		this.configMapLabels = KubernetesUtils.getConfigMapLabels(
+			leaderConfig.getClusterId(), LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+		leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+		FutureUtils.assertNoException(leaderElectorRunFuture);
+		kubernetesWatch = kubeClient.watchConfigMaps(configMapName, new ConfigMapCallbackHandlerImpl());
+		running = true;
+	}
+
+	@Override
+	public void close() {
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+			running = false;
+
+			LOG.info("Closing {}.", this);
+			leaderElectorRunFuture.complete(null);
+			kubernetesWatch.close();
+		}
+	}
+
+	@Override
+	public void writeLeaderInformation(LeaderInformation leaderInformation) {
+		final UUID confirmedLeaderSessionID = leaderInformation.getLeaderSessionID();
+		final String confirmedLeaderAddress = leaderInformation.getLeaderAddress();
+		try {
+			kubeClient.checkAndUpdateConfigMap(
+				configMapName,
+				configMap -> {
+					if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+						// 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) {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("Could not update ConfigMap " + configMapName, e));
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		final Optional<KubernetesConfigMap> configMapOpt = kubeClient.getConfigMap(configMapName);
+		if (configMapOpt.isPresent()) {
+			return KubernetesLeaderElector.hasLeadership(configMapOpt.get(), lockIdentity);
+		} else {
+			fatalErrorHandler.onFatalError(
+				new KubernetesException("ConfigMap " + configMapName + "does not exist.", null));
+			return false;
+		}
+	}
+
+	private class LeaderCallbackHandlerImpl extends KubernetesLeaderElector.LeaderCallbackHandler {
+
+		@Override
+		public void isLeader() {
+			synchronized (lock) {
+				if (running) {
+					leaderElectionEventHandler.onGrantLeadership();
+				}
+			}
+		}
+
+		@Override
+		public void notLeader() {
+			synchronized (lock) {
+				if (running) {
+					// Clear the leader information in ConfigMap
+					try {
+						kubeClient.checkAndUpdateConfigMap(
+							configMapName,
+							configMap -> {
+								if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+									configMap.getData().remove(LEADER_ADDRESS_KEY);
+									configMap.getData().remove(LEADER_SESSION_ID_KEY);
+									return Optional.of(configMap);
+								}
+								return Optional.empty();
+							}
+						).get();
+					} catch (Exception e) {
+						fatalErrorHandler.onFatalError(
+							new LeaderElectionException(
+								"Could not remove leader information from ConfigMap " + configMapName, e));
+					}
+					leaderElectionEventHandler.onRevokeLeadership();
+					// Continue to contend the leader
+					if (!leaderElectorRunFuture.isDone()) {
+						leaderElectorRunFuture.complete(null);
+					}
+					leaderElectorRunFuture = CompletableFuture.runAsync(leaderElector::run, ioExecutor);
+					FutureUtils.assertNoException(leaderElectorRunFuture);
+				}
+			}
+		}
+	}
+
+	private class ConfigMapCallbackHandlerImpl implements FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> {
+		// This is used to get the difference between current and previous data. Since the annotation will be updated
+		// periodically, we need to filter out these noises.
+		final Map<String, String> previousData = new HashMap<>();
+
+		@Override
+		public void onAdded(List<KubernetesConfigMap> configMaps) {
+			// noop
+		}
+
+		@Override
+		public void onModified(List<KubernetesConfigMap> configMaps) {
+			// We should only receive events for the watched ConfigMap
+			final KubernetesConfigMap configMap = checkConfigMaps(configMaps, configMapName);
+
+			if (!configMap.getData().equals(previousData)) {
+				if (KubernetesLeaderElector.hasLeadership(configMap, lockIdentity)) {
+					synchronized (lock) {
+						if (running) {
+							leaderElectionEventHandler.onLeaderInformationChange(
+								getLeaderInformationFromConfigMap(configMap));

Review comment:
       I think the implementor of `onLeaderInformationChange` should check whether the `LeaderInformation` has changed truly. We have already done this in the `DefaultLeaderElectionService`. I will remove the pre-check here and add a description of `LeaderElectionEventHandler#onLeaderInformationChange`. Duplicated leader change event could happen, the implementor should check this.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13644: [FLINK-19542][k8s]Implement LeaderElectionService and LeaderRetrievalService based on Kubernetes API

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13644:
URL: https://github.com/apache/flink/pull/13644#issuecomment-721547397


   @tillrohrmann I have addressed the minor comments and force pushed so that it could be easily merged.
   
   If you want to have a look the changes, you could find the new `fixup`s [here](https://github.com/wangyang0918/flink/commits/FLINK-19542-k8s-ha-leader-election-fixup). These two branch have the same content. I just use the `git rebase -i master` to squash these `fixup` commits.


----------------------------------------------------------------
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:
users@infra.apache.org