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 2021/10/14 12:33:04 UTC

[GitHub] [flink] tillrohrmann opened a new pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

tillrohrmann opened a new pull request #17485:
URL: https://github.com/apache/flink/pull/17485


   I will fill in the description once I have added tests. If you want to try the change out, then set `high-availability: org.apache.flink.kubernetes.highavailability.KubernetesSingleLeaderElectionHaServicesFactory`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsITCase.java
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.UUID;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Integration tests for the {@link ZooKeeperUtils}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperUtilsITCase {
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testWriteAndReadLeaderInformation() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final String path = "/foobar";
+
+        try {
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "barfoo");
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformation readLeaderInformation =
+                    ZooKeeperUtils.readLeaderInformation(
+                            curatorFramework.asCuratorFramework().getData().forPath(path));
+
+            assertThat(readLeaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    @Nonnull
+    private CuratorFrameworkWithUnhandledErrorListener startCuratorFramework() {

Review comment:
       I would leave this for a follow up task to introduce a `CuratorFrameworkExtension`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
##########
@@ -204,6 +204,23 @@
                     .withDescription(
                             "The time before a JobManager after a fail over recovers the current jobs.");
 
+    /**
+     * Safety hatch to fallback to the old ha services implementations.
+     *
+     * <p>Ideally, we can remove this option together with the old implementations in the next
+     * release.
+     */
+    @Documentation.Section(Documentation.Sections.EXPERT_HIGH_AVAILABILITY)

Review comment:
       https://issues.apache.org/jira/browse/FLINK-25806

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
##########
@@ -204,6 +204,23 @@
                     .withDescription(
                             "The time before a JobManager after a fail over recovers the current jobs.");
 
+    /**
+     * Safety hatch to fallback to the old ha services implementations.
+     *
+     * <p>Ideally, we can remove this option together with the old implementations in the next
+     * release.
+     */
+    @Documentation.Section(Documentation.Sections.EXPERT_HIGH_AVAILABILITY)
+    public static final ConfigOption<Boolean> USE_OLD_HA_SERVICES =
+            key("high-availability.use-old-ha-services")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "Use this option to disable the new HA service implementations for ZooKeeper and K8s. This is safety hatch in case that the new ha services are buggy.")

Review comment:
       Will correct 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30248",
       "triggerID" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   * c7aaf594b9767994a95d9cba1c2f7e60d385002a Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30248) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    @GuardedBy("lock")
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = Preconditions.checkNotNull(fatalErrorHandler);
+
+        this.leadershipOperationExecutor = Preconditions.checkNotNull(leadershipOperationExecutor);
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                multipleComponentLeaderElectionDriverFactory,
+                Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(String.format("leadershipOperationExecutor"))));

Review comment:
       Good catch. Will remove 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   * c7aaf594b9767994a95d9cba1c2f7e60d385002a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            getLeaderCallback().notLeader();
+                            leaderElectionListener.await(LeaderElectionEvent.NotLeaderEvent.class);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testPublishLeaderInformation() throws Exception {}
+    public void testPublishLeaderInformation() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            final LeaderInformation leaderInformation =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            final String componentId = "foobar";

Review comment:
       I'll change 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(

Review comment:
       We can also rename it into `createLeaderElectionDriver` if this helps.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();

Review comment:
       True. Since this is unrelated to my changes I will leave this for a future follow up. I fear that making this renamings will screw up the fixing of other changes.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503) 
   * a3c818cb7a4341243bbf7f584d327a1f51d355d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a3c818cb7a4341243bbf7f584d327a1f51d355d9 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155) 
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
##########
@@ -204,6 +204,23 @@
                     .withDescription(
                             "The time before a JobManager after a fail over recovers the current jobs.");
 
+    /**
+     * Safety hatch to fallback to the old ha services implementations.
+     *
+     * <p>Ideally, we can remove this option together with the old implementations in the next
+     * release.
+     */
+    @Documentation.Section(Documentation.Sections.EXPERT_HIGH_AVAILABILITY)

Review comment:
       https://issues.apache.org/jira/browse/FLINK-2580




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] XComp commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java
##########
@@ -78,21 +84,46 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore(
             SharedStateRegistryFactory sharedStateRegistryFactory,
             Executor ioExecutor)
             throws Exception {
+        final String configMapName = getConfigMapNameFunction.apply(jobID);
+        KubernetesUtils.createConfigMapIfItDoesNotExist(kubeClient, configMapName, clusterId);
 
         return KubernetesUtils.createCompletedCheckpointStore(
                 configuration,
                 kubeClient,
                 executor,
-                getConfigMapNameFunction.apply(jobID),
+                configMapName,
                 lockIdentity,
                 maxNumberOfCheckpointsToRetain,
                 sharedStateRegistryFactory,
                 ioExecutor);
     }
 
     @Override
-    public CheckpointIDCounter createCheckpointIDCounter(JobID jobID) {
-        return new KubernetesCheckpointIDCounter(
-                kubeClient, getConfigMapNameFunction.apply(jobID), lockIdentity);
+    public CheckpointIDCounter createCheckpointIDCounter(JobID jobID) throws Exception {

Review comment:
       To me, it would feel more natural to have a subclass of `KubernetesCheckpointRecoveryFactory` that takes care of the ConfigMap creation. But I don't have a strong argument towards refactoring this code because we wouldn't gain much from such a refactoring.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = Preconditions.checkNotNull(curatorFramework);
+        this.leaderElectionListener = Preconditions.checkNotNull(leaderElectionListener);
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchPath());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_UPDATED:
+                                    Preconditions.checkNotNull(
+                                            event.getData(),
+                                            "The ZooKeeper event data must not be null.");
+                                    handleChangedLeaderInformation(event.getData());
+                                    break;
+                                case NODE_REMOVED:
+                                    Preconditions.checkNotNull(
+                                            event.getData(),
+                                            "The ZooKeeper event data must not be null.");
+                                    handleRemovedLeaderInformation(event.getData().getPath());
+                                    break;
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+
+        if (!leaderLatch.hasLeadership()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(
+                curatorFramework, ZooKeeperUtils.generateZookeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 no longer participates in the leader election.");
+                break;
+        }
+    }
+
+    @Override
+    public void isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (shouldHandleLeaderInformationEvent(childData.getPath())) {
+            final String leaderName = extractLeaderName(childData.getPath());
+
+            final LeaderInformation leaderInformation =
+                    tryReadingLeaderInformation(childData, leaderName);
+
+            leaderElectionListener.notifyLeaderInformationChange(leaderName, leaderInformation);
+        }
+    }
+
+    private String extractLeaderName(String path) {
+        final String[] splits = ZooKeeperUtils.splitZooKeeperPath(path);
+
+        Preconditions.checkState(
+                splits.length >= 2,
+                String.format(
+                        "Expecting path consisting of /<leader_name>/connection_info. Got path '%s'",
+                        path));
+
+        return splits[splits.length - 2];
+    }
+
+    private void handleRemovedLeaderInformation(String removedNodePath) {
+        if (shouldHandleLeaderInformationEvent(removedNodePath)) {
+            final String leaderName = extractLeaderName(removedNodePath);
+
+            leaderElectionListener.notifyLeaderInformationChange(
+                    leaderName, LeaderInformation.empty());
+        }
+    }
+
+    private boolean shouldHandleLeaderInformationEvent(String removedNode) {
+        return running.get() && leaderLatch.hasLeadership() && isConnectionInfoNode(removedNode);

Review comment:
       Good catch. Will change 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   * c7aaf594b9767994a95d9cba1c2f7e60d385002a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);
+
+            ExecutorUtils.gracefulShutdown(10L, TimeUnit.SECONDS, leadershipOperationExecutor);
+
+            Exception exception = null;
+            try {
+                multipleComponentLeaderElectionDriver.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public LeaderElectionDriverFactory createDriverFactory(String leaderName) {
+        return new MultipleComponentLeaderElectionDriverAdapterFactory(leaderName, this);
+    }
+
+    @Override
+    public void publishLeaderInformation(String leaderName, LeaderInformation leaderInformation) {
+        try {
+            multipleComponentLeaderElectionDriver.publishLeaderInformation(
+                    leaderName, leaderInformation);
+        } catch (Exception e) {
+            fatalErrorHandler.onFatalError(
+                    new FlinkException(
+                            String.format(
+                                    "Could not write leader information %s for leader %s.",
+                                    leaderInformation, leaderName),
+                            e));
+        }
+    }
+
+    @Override
+    public void registerLeaderElectionEventHandler(
+            String componentId, LeaderElectionEventHandler leaderElectionEventHandler) {
+
+        synchronized (lock) {
+            Preconditions.checkState(
+                    !leaderElectionEventHandlers.containsKey(componentId),
+                    "Do not support duplicate LeaderElectionEventHandler registration under %s",
+                    componentId);
+            leaderElectionEventHandlers.put(componentId, leaderElectionEventHandler);
+
+            if (currentLeaderSessionId != null) {
+                leadershipOperationExecutor.execute(
+                        () -> leaderElectionEventHandler.onGrantLeadership(currentLeaderSessionId));
+            }
+        }
+    }
+
+    @Override
+    public void unregisterLeaderElectionEventHandler(String componentId) throws Exception {
+        final LeaderElectionEventHandler unregisteredLeaderElectionEventHandler;
+        synchronized (lock) {
+            unregisteredLeaderElectionEventHandler =
+                    leaderElectionEventHandlers.remove(componentId);
+
+            if (unregisteredLeaderElectionEventHandler != null) {

Review comment:
       True. Will add it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+/**
+ * Leader election service that allows to register multiple {@link LeaderElectionEventHandler
+ * LeaderElectionEventHandlers} that are identified by different names. For each event handler it is
+ * possible to write the corresponding {@link LeaderInformation}.
+ */
+public interface MultipleComponentLeaderElectionService {

Review comment:
       This is probably a good idea. Will add it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;

Review comment:
       Good catch. Will change it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);

Review comment:
       I think I wanted to put `this.getClass().getSimpleName()` there. Will update it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);
+
+            ExecutorUtils.gracefulShutdown(10L, TimeUnit.SECONDS, leadershipOperationExecutor);
+
+            Exception exception = null;
+            try {
+                multipleComponentLeaderElectionDriver.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            ExceptionUtils.tryRethrowException(exception);

Review comment:
       No, you are right. I will update it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(

Review comment:
       Indeed. Will update it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionDriverFactory.java
##########
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 {@link MultipleComponentLeaderElectionDriver}. */
+public interface MultipleComponentLeaderElectionDriverFactory {
+
+    /**
+     * Creates a {@link MultipleComponentLeaderElectionDriver} for the given leader contender
+     * description. Moreover, it registers the given leader election listener with the service.

Review comment:
       The `MultipleComponentLeaderElectionDriverAdapter` is an implementation of the `LeaderElectionDriver` and not the `MultipleComponentLeaderElectionDriver`. Therefore, I think the comment makes sense. It refers to the fact that the given listener will be registered with the created MCLElectionDriver.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Collection;
+
+/**
+ * A leader election driver that allows to write {@link LeaderInformation} for multiple components.
+ */
+public interface MultipleComponentLeaderElectionDriver {
+
+    /**
+     * Closes the driver.
+     *
+     * @throws Exception if closing this driver fails
+     */
+    void close() throws Exception;
+
+    /**
+     * Returns whether the driver has currently leadership.
+     *
+     * @return {@code true} if the driver has leadership, otherwise {@code false}
+     */
+    boolean hasLeadership();
+
+    /**
+     * Publishes the leader information for the given component.
+     *
+     * @param componentId identifying the component for which to publish the leader information
+     * @param leaderInformation leader information of the respective component
+     * @throws Exception if publishing fails
+     */
+    void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception;
+
+    /**
+     * Deletes the leader information for the given component.
+     *
+     * @param componentId identifying the component for which to delete the leader information
+     * @throws Exception if deleting fails
+     */
+    void deleteLeaderInformation(String componentId) throws Exception;
+
+    /**
+     * Listener interface for state changes of the {@link MultipleComponentLeaderElectionDriver}.
+     */
+    interface Listener {
+
+        /** Callback that is called once the driver obtains the leadership. */
+        void isLeader();
+
+        /** Callback that is called once the driver loses the leadership. */
+        void notLeader();
+
+        /**
+         * Notifies the listener about a changed leader information for the given component.
+         *
+         * @param componentId identifying the component whose leader information has changed
+         * @param leaderInformation new leader information
+         */
+        void notifyLeaderInformationChange(String componentId, LeaderInformation leaderInformation);
+
+        /**
+         * Notifies the listener about all currently known leader information.
+         *
+         * @param leaderInformationWithComponentIds leader information with component ids
+         */
+        void notifyAllKnownLeaderInformation(

Review comment:
       Yes, the different implementations manifest in different interface methods :-(

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);
+
+            ExecutorUtils.gracefulShutdown(10L, TimeUnit.SECONDS, leadershipOperationExecutor);
+
+            Exception exception = null;
+            try {
+                multipleComponentLeaderElectionDriver.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public LeaderElectionDriverFactory createDriverFactory(String leaderName) {
+        return new MultipleComponentLeaderElectionDriverAdapterFactory(leaderName, this);
+    }
+
+    @Override
+    public void publishLeaderInformation(String leaderName, LeaderInformation leaderInformation) {
+        try {
+            multipleComponentLeaderElectionDriver.publishLeaderInformation(
+                    leaderName, leaderInformation);
+        } catch (Exception e) {
+            fatalErrorHandler.onFatalError(
+                    new FlinkException(
+                            String.format(
+                                    "Could not write leader information %s for leader %s.",
+                                    leaderInformation, leaderName),
+                            e));
+        }
+    }
+
+    @Override
+    public void registerLeaderElectionEventHandler(
+            String componentId, LeaderElectionEventHandler leaderElectionEventHandler) {
+
+        synchronized (lock) {
+            Preconditions.checkState(
+                    !leaderElectionEventHandlers.containsKey(componentId),
+                    "Do not support duplicate LeaderElectionEventHandler registration under %s",
+                    componentId);
+            leaderElectionEventHandlers.put(componentId, leaderElectionEventHandler);
+
+            if (currentLeaderSessionId != null) {
+                leadershipOperationExecutor.execute(
+                        () -> leaderElectionEventHandler.onGrantLeadership(currentLeaderSessionId));
+            }
+        }
+    }
+
+    @Override
+    public void unregisterLeaderElectionEventHandler(String componentId) throws Exception {
+        final LeaderElectionEventHandler unregisteredLeaderElectionEventHandler;
+        synchronized (lock) {
+            unregisteredLeaderElectionEventHandler =
+                    leaderElectionEventHandlers.remove(componentId);
+
+            if (unregisteredLeaderElectionEventHandler != null) {
+                leadershipOperationExecutor.execute(
+                        unregisteredLeaderElectionEventHandler::onRevokeLeadership);
+            }
+        }
+
+        multipleComponentLeaderElectionDriver.deleteLeaderInformation(componentId);
+    }
+
+    @Override
+    public boolean hasLeadership(String leaderName) {
+        synchronized (lock) {
+            Preconditions.checkState(running);
+
+            return leaderElectionEventHandlers.containsKey(leaderName)
+                    && multipleComponentLeaderElectionDriver.hasLeadership();
+        }
+    }
+
+    @Override
+    public void isLeader() {
+        final UUID newLeaderSessionId = UUID.randomUUID();
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            currentLeaderSessionId = UUID.randomUUID();
+
+            forEachLeaderElectionEventHandler(
+                    leaderElectionEventHandler ->
+                            leaderElectionEventHandler.onGrantLeadership(newLeaderSessionId));
+        }
+    }
+
+    @Override
+    public void notLeader() {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            currentLeaderSessionId = null;
+
+            forEachLeaderElectionEventHandler(LeaderElectionEventHandler::onRevokeLeadership);
+        }
+    }
+
+    @GuardedBy("lock")
+    private void forEachLeaderElectionEventHandler(
+            Consumer<? super LeaderElectionEventHandler> action) {
+
+        for (LeaderElectionEventHandler leaderElectionEventHandler :
+                leaderElectionEventHandlers.values()) {
+            leadershipOperationExecutor.execute(() -> action.accept(leaderElectionEventHandler));
+        }
+    }
+
+    @Override
+    public void notifyLeaderInformationChange(
+            String leaderName, LeaderInformation leaderInformation) {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            final LeaderElectionEventHandler leaderElectionEventHandler =
+                    leaderElectionEventHandlers.get(leaderName);
+
+            if (leaderElectionEventHandler != null) {
+                leadershipOperationExecutor.execute(
+                        () ->
+                                leaderElectionEventHandler.onLeaderInformationChange(
+                                        leaderInformation));
+            }
+        }
+    }
+
+    @Override
+    public void notifyAllKnownLeaderInformation(
+            Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            final Map<String, LeaderInformation> leaderInformationByName =
+                    leaderInformationWithComponentIds.stream()
+                            .collect(
+                                    Collectors.toMap(
+                                            LeaderInformationWithComponentId::getComponentId,
+                                            LeaderInformationWithComponentId
+                                                    ::getLeaderInformation));
+
+            for (Map.Entry<String, LeaderElectionEventHandler>
+                    leaderNameLeaderElectionEventHandlerPair :
+                            leaderElectionEventHandlers.entrySet()) {
+                final String leaderName = leaderNameLeaderElectionEventHandlerPair.getKey();
+                if (leaderInformationByName.containsKey(leaderName)) {
+                    leaderNameLeaderElectionEventHandlerPair
+                            .getValue()
+                            .onLeaderInformationChange(leaderInformationByName.get(leaderName));

Review comment:
       I think we filter out these cases on a higher level.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingMultipleComponentLeaderElectionDriver.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.runtime.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.util.Optional;
+
+/** Testing implementation of {@link MultipleComponentLeaderElectionDriver}. */
+public class TestingMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private final BiConsumerWithException<String, LeaderInformation, Exception>
+            publishLeaderInformationConsumer;
+    private final ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer;
+    private boolean hasLeadership;
+
+    private Optional<Listener> listener;
+
+    private TestingMultipleComponentLeaderElectionDriver(
+            BiConsumerWithException<String, LeaderInformation, Exception>
+                    publishLeaderInformationConsumer,
+            ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer) {
+        this.publishLeaderInformationConsumer = publishLeaderInformationConsumer;
+        this.deleteLeaderInformationConsumer = deleteLeaderInformationConsumer;
+        hasLeadership = false;
+        listener = Optional.empty();
+    }
+
+    public void grantLeadership() {
+        if (!hasLeadership) {
+            hasLeadership = true;
+            listener.ifPresent(Listener::isLeader);
+        }
+    }
+
+    public void revokeLeadership() {
+        if (hasLeadership) {
+            hasLeadership = false;
+            listener.ifPresent(Listener::notLeader);
+        }
+    }
+
+    public void setListener(Listener listener) {
+        Preconditions.checkState(!this.listener.isPresent(), "Can only set a single listener.");
+        this.listener = Optional.of(listener);
+    }
+
+    @Override
+    public void close() throws Exception {}
+
+    @Override
+    public boolean hasLeadership() {
+        return hasLeadership;
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        publishLeaderInformationConsumer.accept(componentId, leaderInformation);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String componentId) throws Exception {
+        deleteLeaderInformationConsumer.accept(componentId);
+    }
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    public static final class Builder {
+        private BiConsumerWithException<String, LeaderInformation, Exception>
+                publishLeaderInformationConsumer = (ignoredA, ignoredB) -> {};
+        private ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer =
+                (ignored) -> {};

Review comment:
       True. Will update it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingMultipleComponentLeaderElectionDriver.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.runtime.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.util.Optional;
+
+/** Testing implementation of {@link MultipleComponentLeaderElectionDriver}. */
+public class TestingMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private final BiConsumerWithException<String, LeaderInformation, Exception>
+            publishLeaderInformationConsumer;
+    private final ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer;
+    private boolean hasLeadership;
+
+    private Optional<Listener> listener;

Review comment:
       I think this is because of your inspection settings.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull

Review comment:
       Yes, I will correct it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull
+    private DefaultMultipleComponentLeaderElectionService
+            createDefaultMultiplexingLeaderElectionService(
+                    TestingMultipleComponentLeaderElectionDriver leaderElectionDriver)
+                    throws Exception {
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =

Review comment:
       Yes, I will update the code.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull
+    private DefaultMultipleComponentLeaderElectionService
+            createDefaultMultiplexingLeaderElectionService(
+                    TestingMultipleComponentLeaderElectionDriver leaderElectionDriver)
+                    throws Exception {
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                new DefaultMultipleComponentLeaderElectionService(
+                        fatalErrorHandlerExtension.getTestingFatalErrorHandler(),
+                        "foobar",
+                        new TestingMultipleComponentLeaderElectionDriverFactory(
+                                leaderElectionDriver),
+                        Executors.newDirectExecutorService());
+        return leaderElectionService;
+    }
+
+    @Test
+    public void notLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+            leaderElectionDriver.revokeLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isFalse();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void unregisteredEventHandlersAreNotNotified() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            final String componentId = "foobar";
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    componentId, leaderElectionEventHandler);
+            leaderElectionService.unregisterLeaderElectionEventHandler(componentId);
+
+            leaderElectionDriver.grantLeadership();
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isFalse();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void newlyRegisteredEventHandlersAreInformedAboutLeadership() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            leaderElectionDriver.grantLeadership();
+
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    "foobar", leaderElectionEventHandler);
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isTrue();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void allKnownLeaderInformationCallsEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            leaderElectionDriver.grantLeadership();
+
+            final Collection<Component> knownLeaderInformation = createComponents(3);
+            final Collection<Component> unknownLeaderInformation = createComponents(2);

Review comment:
       I think I wanted to also register these components. Just not send their leader information in the `notifyAllKnownLeaderInformation`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull
+    private DefaultMultipleComponentLeaderElectionService
+            createDefaultMultiplexingLeaderElectionService(
+                    TestingMultipleComponentLeaderElectionDriver leaderElectionDriver)
+                    throws Exception {
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                new DefaultMultipleComponentLeaderElectionService(
+                        fatalErrorHandlerExtension.getTestingFatalErrorHandler(),
+                        "foobar",
+                        new TestingMultipleComponentLeaderElectionDriverFactory(
+                                leaderElectionDriver),
+                        Executors.newDirectExecutorService());
+        return leaderElectionService;
+    }
+
+    @Test
+    public void notLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+            leaderElectionDriver.revokeLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isFalse();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void unregisteredEventHandlersAreNotNotified() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            final String componentId = "foobar";
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    componentId, leaderElectionEventHandler);
+            leaderElectionService.unregisterLeaderElectionEventHandler(componentId);
+
+            leaderElectionDriver.grantLeadership();
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isFalse();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void newlyRegisteredEventHandlersAreInformedAboutLeadership() throws Exception {

Review comment:
       I am not sure whether this is really needed. I think it is fine to leave this undefined for the time being.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {
+        // Maybe use a single service for leader retrieval
+        return ZooKeeperUtils.createLeaderRetrievalService(
+                leaderNamespacedCuratorFramework, leaderPath, configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        try {
+            super.internalClose();
+        } catch (Exception e) {
+            exception = ExceptionUtils.firstOrSuppressed(e, exception);
+        }
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        super.internalCleanupJobData(jobID);
+    }

Review comment:
       True. I will remove it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {

Review comment:
       Yes and probably not. Will change it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+
+import javax.annotation.Nonnull;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Abstract ZooKeeper based HA services. */
+public abstract class AbstractZooKeeperHaServices extends AbstractHaServices {
+    /** The curator resource to use. */
+    private final CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper;
+
+    public AbstractZooKeeperHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Executor executor,
+            Configuration configuration,
+            BlobStoreService blobStoreService) {
+        super(configuration, executor, blobStoreService);
+        this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper);
+    }
+
+    protected final CuratorFramework getCuratorFramework() {
+        return curatorFrameworkWrapper.asCuratorFramework();
+    }
+
+    @Override
+    public CheckpointRecoveryFactory createCheckpointRecoveryFactory() throws Exception {
+        return new ZooKeeperCheckpointRecoveryFactory(
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        curatorFrameworkWrapper.asCuratorFramework(), ZooKeeperUtils.getJobsPath()),
+                configuration,
+                ioExecutor);
+    }
+
+    @Override
+    public JobGraphStore createJobGraphStore() throws Exception {
+        return ZooKeeperUtils.createJobGraphs(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    public RunningJobsRegistry createRunningJobsRegistry() {
+        return new ZooKeeperRunningJobsRegistry(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        curatorFrameworkWrapper.close();
+    }
+
+    @Override
+    protected void internalCleanup() throws Exception {
+        cleanupZooKeeperPaths();
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        deleteZNode(ZooKeeperUtils.getLeaderPathForJob(jobID));
+    }
+
+    /** Cleans up leftover ZooKeeper paths. */
+    private void cleanupZooKeeperPaths() throws Exception {
+        deleteOwnedZNode();
+        tryDeleteEmptyParentZNodes();
+    }
+
+    private void deleteOwnedZNode() throws Exception {
+        deleteZNode("/");
+    }
+
+    protected void deleteZNode(String path) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFrameworkWrapper.asCuratorFramework(), path);
+    }
+
+    /**
+     * Tries to delete empty parent znodes.
+     *
+     * <p>IMPORTANT: This method can be removed once all supported ZooKeeper versions support the
+     * container {@link org.apache.zookeeper.CreateMode}.
+     *
+     * @throws Exception if the deletion fails for other reason than {@link
+     *     KeeperException.NotEmptyException}
+     */
+    private void tryDeleteEmptyParentZNodes() throws Exception {
+        // try to delete the parent znodes if they are empty
+        String remainingPath =
+                getParentPath(
+                        getNormalizedPath(
+                                curatorFrameworkWrapper.asCuratorFramework().getNamespace()));
+        final CuratorFramework nonNamespaceClient =
+                curatorFrameworkWrapper.asCuratorFramework().usingNamespace(null);
+
+        while (!isRootPath(remainingPath)) {
+            try {
+                nonNamespaceClient.delete().forPath(remainingPath);
+            } catch (KeeperException.NotEmptyException ignored) {
+                // We can only delete empty znodes
+                break;
+            }
+
+            remainingPath = getParentPath(remainingPath);
+        }
+    }
+
+    private static boolean isRootPath(String remainingPath) {
+        return ZKPaths.PATH_SEPARATOR.equals(remainingPath);
+    }
+
+    @Nonnull

Review comment:
       Will remove it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+
+import javax.annotation.Nonnull;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Abstract ZooKeeper based HA services. */
+public abstract class AbstractZooKeeperHaServices extends AbstractHaServices {
+    /** The curator resource to use. */
+    private final CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper;
+
+    public AbstractZooKeeperHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Executor executor,
+            Configuration configuration,
+            BlobStoreService blobStoreService) {
+        super(configuration, executor, blobStoreService);
+        this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper);
+    }
+
+    protected final CuratorFramework getCuratorFramework() {
+        return curatorFrameworkWrapper.asCuratorFramework();
+    }
+
+    @Override
+    public CheckpointRecoveryFactory createCheckpointRecoveryFactory() throws Exception {
+        return new ZooKeeperCheckpointRecoveryFactory(
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        curatorFrameworkWrapper.asCuratorFramework(), ZooKeeperUtils.getJobsPath()),
+                configuration,
+                ioExecutor);
+    }
+
+    @Override
+    public JobGraphStore createJobGraphStore() throws Exception {
+        return ZooKeeperUtils.createJobGraphs(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    public RunningJobsRegistry createRunningJobsRegistry() {
+        return new ZooKeeperRunningJobsRegistry(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        curatorFrameworkWrapper.close();
+    }
+
+    @Override
+    protected void internalCleanup() throws Exception {
+        cleanupZooKeeperPaths();
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        deleteZNode(ZooKeeperUtils.getLeaderPathForJob(jobID));
+    }
+
+    /** Cleans up leftover ZooKeeper paths. */
+    private void cleanupZooKeeperPaths() throws Exception {
+        deleteOwnedZNode();
+        tryDeleteEmptyParentZNodes();
+    }
+
+    private void deleteOwnedZNode() throws Exception {
+        deleteZNode("/");
+    }
+
+    protected void deleteZNode(String path) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFrameworkWrapper.asCuratorFramework(), path);
+    }
+
+    /**
+     * Tries to delete empty parent znodes.
+     *
+     * <p>IMPORTANT: This method can be removed once all supported ZooKeeper versions support the
+     * container {@link org.apache.zookeeper.CreateMode}.
+     *
+     * @throws Exception if the deletion fails for other reason than {@link
+     *     KeeperException.NotEmptyException}
+     */
+    private void tryDeleteEmptyParentZNodes() throws Exception {
+        // try to delete the parent znodes if they are empty
+        String remainingPath =
+                getParentPath(
+                        getNormalizedPath(
+                                curatorFrameworkWrapper.asCuratorFramework().getNamespace()));
+        final CuratorFramework nonNamespaceClient =
+                curatorFrameworkWrapper.asCuratorFramework().usingNamespace(null);
+
+        while (!isRootPath(remainingPath)) {
+            try {
+                nonNamespaceClient.delete().forPath(remainingPath);
+            } catch (KeeperException.NotEmptyException ignored) {
+                // We can only delete empty znodes
+                break;
+            }
+
+            remainingPath = getParentPath(remainingPath);
+        }
+    }
+
+    private static boolean isRootPath(String remainingPath) {
+        return ZKPaths.PATH_SEPARATOR.equals(remainingPath);
+    }
+
+    @Nonnull
+    private static String getNormalizedPath(String path) {
+        return ZKPaths.makePath(path, "");
+    }
+
+    @Nonnull

Review comment:
       Will remove it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(

Review comment:
       Yes, the `fatalErrorHandler` is used for asynchronous exception that might occur. When doing synchronous call, I think it is better to directly throw the exception so that the caller can handle it appropriately.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {

Review comment:
       Yes, I think what we would save by moving it to the `AbstractZooKeeperHaServices` does not offset the potential costs that the leader election and retrieval service are not close together which makes it easier to keep them in sync.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -96,13 +106,17 @@
     /** The prefix of the completed checkpoint file. */
     public static final String HA_STORAGE_COMPLETED_CHECKPOINT = "completedCheckpoint";
 
-    private static final String RESOURCE_MANAGER_LEADER = "/resource_manager";
+    private static final String RESOURCE_MANAGER_LEADER = "resource_manager";
 
-    private static final String DISPATCHER_LEADER = "/dispatcher";
+    private static final String DISPATCHER_LEADER = "dispatcher";
 
     private static final String LEADER_NODE = "/leader";

Review comment:
       True. This is a nice unification. Will apply it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -96,13 +106,17 @@
     /** The prefix of the completed checkpoint file. */
     public static final String HA_STORAGE_COMPLETED_CHECKPOINT = "completedCheckpoint";
 
-    private static final String RESOURCE_MANAGER_LEADER = "/resource_manager";
+    private static final String RESOURCE_MANAGER_LEADER = "resource_manager";
 
-    private static final String DISPATCHER_LEADER = "/dispatcher";
+    private static final String DISPATCHER_LEADER = "dispatcher";
 
     private static final String LEADER_NODE = "/leader";
 
-    private static final String REST_SERVER_LEADER = "/rest_server";
+    private static final String REST_SERVER_LEADER = "rest_server";
+
+    private static final String LEADER_LATCH_NODE = "/latch";

Review comment:
       Yes. Will update it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {
+        // Maybe use a single service for leader retrieval
+        return ZooKeeperUtils.createLeaderRetrievalService(
+                leaderNamespacedCuratorFramework, leaderPath, configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        try {
+            super.internalClose();
+        } catch (Exception e) {
+            exception = ExceptionUtils.firstOrSuppressed(e, exception);
+        }
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        super.internalCleanupJobData(jobID);
+    }
+
+    @Override
+    protected String getLeaderPathForResourceManager() {
+        return ZooKeeperUtils.getResourceManagerNode();
+    }
+
+    @Override
+    protected String getLeaderPathForDispatcher() {
+        return ZooKeeperUtils.getDispatcherNode();
+    }
+
+    @Override
+    protected String getLeaderPathForJobManager(JobID jobID) {
+        return jobID.toString();
+    }
+
+    @Override
+    protected String getLeaderPathForRestServer() {
+        return ZooKeeperUtils.getRestServerNode();
+    }

Review comment:
       I think you are right that there is a difference between the method name and what is returned. In this case I would suggest to change the naming of the methods to `getLeaderNameXYZ` instead of `getLeaderPathXZY`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {
+        // Maybe use a single service for leader retrieval
+        return ZooKeeperUtils.createLeaderRetrievalService(
+                leaderNamespacedCuratorFramework, leaderPath, configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        try {
+            super.internalClose();
+        } catch (Exception e) {
+            exception = ExceptionUtils.firstOrSuppressed(e, exception);
+        }
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        super.internalCleanupJobData(jobID);
+    }
+
+    @Override
+    protected String getLeaderPathForResourceManager() {
+        return ZooKeeperUtils.getResourceManagerNode();
+    }
+
+    @Override
+    protected String getLeaderPathForDispatcher() {
+        return ZooKeeperUtils.getDispatcherNode();
+    }
+
+    @Override
+    protected String getLeaderPathForJobManager(JobID jobID) {
+        return jobID.toString();
+    }
+
+    @Override
+    protected String getLeaderPathForRestServer() {
+        return ZooKeeperUtils.getRestServerNode();
+    }

Review comment:
       Arg, the problem is that the different `AbstractHaServices` implementations treat this method differently. Since I don't want to touch the `ZooKeeperHaServices`, I will keep it as is. Once `ZooKeeperHaServices` is removed, we can clean it up.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -96,13 +106,17 @@
     /** The prefix of the completed checkpoint file. */
     public static final String HA_STORAGE_COMPLETED_CHECKPOINT = "completedCheckpoint";
 
-    private static final String RESOURCE_MANAGER_LEADER = "/resource_manager";
+    private static final String RESOURCE_MANAGER_LEADER = "resource_manager";

Review comment:
       This has been addressed in a previous fixup.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -137,17 +155,37 @@ public static String getCheckpointIdCounterPath() {
         return "/checkpoint_id_counter";
     }
 
+    public static String getLeaderPath() {
+        return LEADER_NODE;
+    }
+
+    public static String getDispatcherNode() {
+        return DISPATCHER_LEADER;
+    }
+
+    public static String getResourceManagerNode() {
+        return RESOURCE_MANAGER_LEADER;
+    }
+
+    public static String getRestServerNode() {
+        return REST_SERVER_LEADER;
+    }
+
+    public static String getLeaderLatchNode() {
+        return LEADER_LATCH_NODE;
+    }
+
     private static String getLeaderPath(String suffix) {
         return generateZookeeperPath(LEADER_NODE, suffix);
     }
 
     @Nonnull

Review comment:
       True, but I haven't touched this code in this PR. Can be a follow-up task.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -555,6 +668,10 @@ public static String generateZookeeperPath(String root, String child) {
         return result;
     }
 
+    public static String makeZooKeeperPath(String path) {

Review comment:
       True, I will change it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;

Review comment:
       I am a bit torn here. The general Flink guideline says that arguments that are not annotated with `@Nullable` can be assumed to be non-null. Hence, a null check should not strictly be necessary. On the other hand, failing fast in case of a NPE is always a good idea.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (running.get() && leaderLatch.hasLeadership() && isConnectionInfoNode(childData)) {
+
+            final String path = childData.getPath();
+            final String[] splits = path.split("/");
+
+            Preconditions.checkState(
+                    splits.length >= 2,
+                    String.format(
+                            "Expecting path consisting of <leader_name>/connection_info. Got path '%s'",
+                            path));
+            final String leaderName = splits[splits.length - 2];
+
+            final LeaderInformation leaderInformation =
+                    tryReadingLeaderInformation(childData, leaderName);
+
+            leaderElectionListener.notifyLeaderInformationChange(leaderName, leaderInformation);
+        }
+    }
+
+    private boolean isConnectionInfoNode(ChildData childData) {
+        return childData.getPath().endsWith(ZooKeeperUtils.CONNECTION_INFO_NODE);
+    }
+
+    private LeaderInformation tryReadingLeaderInformation(ChildData childData, String id) {
+        LeaderInformation leaderInformation;
+        try {
+            leaderInformation = ZooKeeperUtils.readLeaderInformation(childData.getData());
+
+            LOG.debug("Leader information for {} has changed to {}.", id, leaderInformation);
+        } catch (IOException | ClassNotFoundException e) {
+            LOG.debug(
+                    "Could not read leader information for {}. Rewriting the information.", id, e);
+            leaderInformation = LeaderInformation.empty();
+        }
+
+        return leaderInformation;
+    }
+
+    private static class ConnectionInfoNodeSelector implements TreeCacheSelector {

Review comment:
       Will do.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (running.get() && leaderLatch.hasLeadership() && isConnectionInfoNode(childData)) {
+
+            final String path = childData.getPath();
+            final String[] splits = path.split("/");

Review comment:
       Agreed for the splitting. The leaderName retrieval is specific to the driver and the used ha services. That's why I would like to keep it here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {

Review comment:
       Yes, I think you are right that this should be a `checkState` for the `event.getData() != null`. Moreover, I think that I am handling the `NODE_REMOVED` event wrongly. In this case, the leader information should be marked as `LeaderInformation.empty` because the event will be created with the old node data. I will correct the code.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    private void handleStateChange(ConnectionState newState) {

Review comment:
       I would like to keep it because the `ZooKeeperLeaderElectionDriver` should ideally be removed with the next release.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                new ZooKeeperMultipleComponentLeaderElectionDriver(
+                        curatorFramework, "foobar", leaderElectionListener);
+        return leaderElectionDriver;
+    }
+
+    @Nonnull

Review comment:
       Will remove it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull

Review comment:
       Will remove it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                new ZooKeeperMultipleComponentLeaderElectionDriver(
+                        curatorFramework, "foobar", leaderElectionListener);
+        return leaderElectionDriver;
+    }
+
+    @Nonnull
+    private CuratorFrameworkWithUnhandledErrorListener startCuratorFramework() {

Review comment:
       For compositional reasons we might also introduce a `CuratorFrameworkExtension`. I would leave this for a follow up task since it wouldn't change anything for this PR.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =

Review comment:
       True, will remove it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                new ZooKeeperMultipleComponentLeaderElectionDriver(
+                        curatorFramework, "foobar", leaderElectionListener);
+        return leaderElectionDriver;
+    }
+
+    @Nonnull
+    private CuratorFrameworkWithUnhandledErrorListener startCuratorFramework() {
+        final Configuration configuration = new Configuration();
+        configuration.set(
+                HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperExtension.getConnectString());
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework =

Review comment:
       Will correct it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();

Review comment:
       What would be the benefit? Unless there is something, I'd like to keep it as is.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(

Review comment:
       We can also rename it into `createLeaderElectionDriver` if this helps.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {

Review comment:
       Yes. I can add the suffix if this makes things clearer.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();

Review comment:
       True. Since this is unrelated to my changes I will leave this for a future follow up. I fear that making this renamings will screw up the fixing of other changes.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";

Review comment:
       Will change it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {

Review comment:
       Yes, I think we should also write `LeaderInformation.empty()`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {

Review comment:
       I'll add `testPublishEmptyLeaderInformation` and `testNonLeaderCannotPublishLeaderInformation`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";

Review comment:
       Fair enough. I'll change it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());

Review comment:
       Let me check.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));

Review comment:
       Will change it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsITCase.java
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.UUID;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Integration tests for the {@link ZooKeeperUtils}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperUtilsITCase {
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testWriteAndReadLeaderInformation() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final String path = "/foobar";
+
+        try {
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "barfoo");
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformation readLeaderInformation =
+                    ZooKeeperUtils.readLeaderInformation(
+                            curatorFramework.asCuratorFramework().getData().forPath(path));
+
+            assertThat(readLeaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    @Nonnull
+    private CuratorFrameworkWithUnhandledErrorListener startCuratorFramework() {

Review comment:
       I would leave this for a follow up task to introduce a `CuratorFrameworkExtension`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionEvent.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.Collection;
+
+/** Leader election event. */
+public abstract class LeaderElectionEvent {
+    public boolean isIsLeaderEvent() {
+        return false;
+    }
+
+    public boolean isNotLeaderEvent() {
+        return false;
+    }
+
+    public boolean isLeaderInformationChangeEvent() {
+        return false;
+    }
+
+    public boolean isAllKnownLeaderInformationEvent() {
+        return false;
+    }
+
+    public IsLeaderEvent asIsLeaderEvent() {
+        return as(IsLeaderEvent.class);
+    }
+
+    public NotLeaderEvent asNotLeaderEvent() {
+        return as(NotLeaderEvent.class);
+    }
+
+    public LeaderInformationChangeEvent asLeaderInformationChangeEvent() {
+        return as(LeaderInformationChangeEvent.class);
+    }
+
+    public AllKnownLeaderInformationEvent asAllKnownLeaderInformationEvent() {
+        return as(AllKnownLeaderInformationEvent.class);
+    }

Review comment:
       True. I'll remove them.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';

Review comment:
       Yes, this is a good idea.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';

Review comment:
       Let's stick to the existing conventions. I'll move it up.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';
+
+    public static String encodeLeaderInformation(LeaderInformation leaderInformation) {
+        Preconditions.checkState(!leaderInformation.isEmpty());

Review comment:
       True. I will change it.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';
+
+    public static String encodeLeaderInformation(LeaderInformation leaderInformation) {
+        Preconditions.checkState(!leaderInformation.isEmpty());
+        return leaderInformation.getLeaderSessionID().toString()
+                + LEADER_INFORMATION_SEPARATOR
+                + leaderInformation.getLeaderAddress();
+    }
+
+    public static LeaderInformation parseLeaderInformationSafely(String value) {

Review comment:
       I think this does not make a huge difference. But can do.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';
+
+    public static String encodeLeaderInformation(LeaderInformation leaderInformation) {
+        Preconditions.checkState(!leaderInformation.isEmpty());
+        return leaderInformation.getLeaderSessionID().toString()
+                + LEADER_INFORMATION_SEPARATOR
+                + leaderInformation.getLeaderAddress();
+    }
+
+    public static LeaderInformation parseLeaderInformationSafely(String value) {
+        try {
+            return parseLeaderInformation(value);
+        } catch (Throwable throwable) {
+            LOG.debug("Could not parse value {} into LeaderInformation.", value, throwable);
+            return LeaderInformation.empty();
+        }
+    }
+
+    private static LeaderInformation parseLeaderInformation(String value) {
+        final int splitIndex = value.indexOf(LEADER_INFORMATION_SEPARATOR);
+
+        Preconditions.checkState(splitIndex >= 0, "Expecting '<session_id>,<leader_address>'");
+
+        final UUID leaderSessionId = UUID.fromString(value.substring(0, splitIndex));
+        final String leaderAddress = value.substring(splitIndex + 1);
+
+        return LeaderInformation.known(leaderSessionId, leaderAddress);
+    }
+
+    @Nonnull
+    public static String createSingleLeaderKey(String componentId) {
+        return LEADER_PREFIX + componentId;
+    }
+
+    public static boolean isSingleLeaderKey(String key) {
+        return key.startsWith(LEADER_PREFIX);
+    }
+
+    @Nonnull

Review comment:
       I don't think so. Will remove them.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",

Review comment:
       Let me try to remove the leader description.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher;
+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.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.LeaderInformationWithComponentId;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriver;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+
+/** {@link MultipleComponentLeaderElectionDriver} for Kubernetes. */
+public class KubernetesMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(KubernetesMultipleComponentLeaderElectionDriver.class);
+
+    private final FlinkKubeClient kubeClient;
+
+    private final String configMapName;
+
+    private final String lockIdentity;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final KubernetesLeaderElector leaderElector;
+
+    // Labels will be used to clean up the ha related ConfigMaps.
+    private final Map<String, String> configMapLabels;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    private final KubernetesSharedWatcher.Watch kubernetesWatch;
+
+    private AtomicBoolean running = new AtomicBoolean(true);

Review comment:
       True. Will update it.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher;
+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.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.LeaderInformationWithComponentId;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriver;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+
+/** {@link MultipleComponentLeaderElectionDriver} for Kubernetes. */
+public class KubernetesMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(KubernetesMultipleComponentLeaderElectionDriver.class);
+
+    private final FlinkKubeClient kubeClient;
+
+    private final String configMapName;
+
+    private final String lockIdentity;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final KubernetesLeaderElector leaderElector;
+
+    // Labels will be used to clean up the ha related ConfigMaps.
+    private final Map<String, String> configMapLabels;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    private final KubernetesSharedWatcher.Watch kubernetesWatch;
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public KubernetesMultipleComponentLeaderElectionDriver(
+            KubernetesLeaderElectionConfiguration leaderElectionConfiguration,
+            FlinkKubeClient kubeClient,
+            Listener leaderElectionListener,
+            KubernetesConfigMapSharedWatcher configMapSharedWatcher,
+            Executor watchExecutor,
+            FatalErrorHandler fatalErrorHandler) {
+        this.kubeClient = kubeClient;
+        this.leaderElectionListener = leaderElectionListener;
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.configMapName = leaderElectionConfiguration.getConfigMapName();
+        this.lockIdentity = leaderElectionConfiguration.getLockIdentity();
+
+        this.leaderElector =
+                kubeClient.createLeaderElector(
+                        leaderElectionConfiguration, new LeaderCallbackHandlerImpl());
+
+        this.configMapLabels =
+                KubernetesUtils.getConfigMapLabels(
+                        leaderElectionConfiguration.getClusterId(),
+                        LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+        kubernetesWatch =
+                configMapSharedWatcher.watch(
+                        configMapName, new ConfigMapCallbackHandlerImpl(), watchExecutor);
+
+        leaderElector.run();

Review comment:
       I don't see a strong benefit other than complicating the interface. Therefore, I would like to not do it at this point in time.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher;
+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.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.LeaderInformationWithComponentId;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriver;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+
+/** {@link MultipleComponentLeaderElectionDriver} for Kubernetes. */
+public class KubernetesMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(KubernetesMultipleComponentLeaderElectionDriver.class);
+
+    private final FlinkKubeClient kubeClient;
+
+    private final String configMapName;
+
+    private final String lockIdentity;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final KubernetesLeaderElector leaderElector;
+
+    // Labels will be used to clean up the ha related ConfigMaps.
+    private final Map<String, String> configMapLabels;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    private final KubernetesSharedWatcher.Watch kubernetesWatch;
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public KubernetesMultipleComponentLeaderElectionDriver(
+            KubernetesLeaderElectionConfiguration leaderElectionConfiguration,
+            FlinkKubeClient kubeClient,
+            Listener leaderElectionListener,
+            KubernetesConfigMapSharedWatcher configMapSharedWatcher,
+            Executor watchExecutor,
+            FatalErrorHandler fatalErrorHandler) {
+        this.kubeClient = kubeClient;
+        this.leaderElectionListener = leaderElectionListener;
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.configMapName = leaderElectionConfiguration.getConfigMapName();
+        this.lockIdentity = leaderElectionConfiguration.getLockIdentity();
+
+        this.leaderElector =
+                kubeClient.createLeaderElector(
+                        leaderElectionConfiguration, new LeaderCallbackHandlerImpl());
+
+        this.configMapLabels =
+                KubernetesUtils.getConfigMapLabels(
+                        leaderElectionConfiguration.getClusterId(),
+                        LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+        kubernetesWatch =
+                configMapSharedWatcher.watch(
+                        configMapName, new ConfigMapCallbackHandlerImpl(), watchExecutor);
+
+        leaderElector.run();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            leaderElector.stop();
+            kubernetesWatch.close();
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        Preconditions.checkState(running.get());
+        final Optional<KubernetesConfigMap> optionalConfigMap =
+                kubeClient.getConfigMap(configMapName);
+
+        if (optionalConfigMap.isPresent()) {
+            return KubernetesLeaderElector.hasLeadership(optionalConfigMap.get(), lockIdentity);
+        } else {
+            fatalErrorHandler.onFatalError(
+                    new KubernetesException(
+                            String.format(
+                                    "ConfigMap %s does not exist. This indicates that somebody has interfered with Flink's operation.",
+                                    configMapName)));
+            return false;
+        }
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        kubeClient
+                .checkAndUpdateConfigMap(
+                        configMapName,
+                        updateConfigMapWithLeaderInformation(componentId, leaderInformation))
+                .get();
+
+        LOG.debug(
+                "Successfully wrote leader information {} for leader {} into the config map {}.",
+                leaderInformation,
+                componentId,
+                configMapName);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String componentId) throws Exception {
+        publishLeaderInformation(componentId, LeaderInformation.empty());
+    }
+
+    private Function<KubernetesConfigMap, Optional<KubernetesConfigMap>>
+            updateConfigMapWithLeaderInformation(
+                    String leaderName, LeaderInformation leaderInformation) {
+        final String configMapDataKey = KubernetesUtils.createSingleLeaderKey(leaderName);
+
+        return kubernetesConfigMap -> {
+            if (KubernetesLeaderElector.hasLeadership(kubernetesConfigMap, lockIdentity)) {
+                final Map<String, String> data = kubernetesConfigMap.getData();
+
+                if (leaderInformation.isEmpty()) {
+                    data.remove(configMapDataKey);
+                } else {
+                    data.put(
+                            configMapDataKey,
+                            KubernetesUtils.encodeLeaderInformation(leaderInformation));
+                }
+
+                kubernetesConfigMap.getLabels().putAll(configMapLabels);
+                return Optional.of(kubernetesConfigMap);
+            }
+
+            return Optional.empty();
+        };
+    }
+
+    private static Collection<LeaderInformationWithComponentId> extractLeaderInformation(
+            KubernetesConfigMap configMap) {
+        final Map<String, String> data = configMap.getData();
+
+        final Collection<LeaderInformationWithComponentId> leaderInformationWithLeaderNames =
+                new ArrayList<>();
+
+        for (Map.Entry<String, String> keyValuePair : data.entrySet()) {
+            final String key = keyValuePair.getKey();
+            if (KubernetesUtils.isSingleLeaderKey(key)) {
+                final String leaderName = KubernetesUtils.extractLeaderName(key);
+                final LeaderInformation leaderInformation =
+                        KubernetesUtils.parseLeaderInformationSafely(keyValuePair.getValue());

Review comment:
       I think this is ok because the corresponding `DefaultLeaderElectionService` would issue another publish command if it is the leader and its leader information is empty.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.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.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.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesConfigMapSharedWatcher;
+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.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+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.NAME_SEPARATOR;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Kubernetes HA services that use a single leader election service per JobManager. */
+public class KubernetesMultipleComponentLeaderElectionHaServices extends AbstractHaServices {
+
+    private final Object lock = new Object();
+
+    private final String clusterId;
+
+    private final FlinkKubeClient kubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+    private final ExecutorService watchExecutorService;
+
+    private final String lockIdentity;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private DefaultMultipleComponentLeaderElectionService multipleComponentLeaderElectionService =
+            null;
+
+    KubernetesMultipleComponentLeaderElectionHaServices(
+            FlinkKubeClient kubeClient,
+            Executor executor,
+            Configuration config,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler) {
+
+        super(config, executor, blobStoreService);
+        this.kubeClient = checkNotNull(kubeClient);
+        this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID));
+        this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+        this.configMapSharedWatcher =
+                this.kubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+        this.watchExecutorService =
+                Executors.newCachedThreadPool(
+                        new ExecutorThreadFactory("config-map-watch-handler"));
+
+        lockIdentity = UUID.randomUUID().toString();
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService =
+                getOrInitializeSingleLeaderElectionService();
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    private DefaultMultipleComponentLeaderElectionService
+            getOrInitializeSingleLeaderElectionService() {
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService == null) {
+                try {
+
+                    final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
+                            new KubernetesLeaderElectionConfiguration(
+                                    getClusterConfigMap(), lockIdentity, configuration);
+                    multipleComponentLeaderElectionService =
+                            new DefaultMultipleComponentLeaderElectionService(
+                                    fatalErrorHandler,
+                                    "Single leader election service",
+                                    new KubernetesMultipleComponentLeaderElectionDriverFactory(
+                                            kubeClient,
+                                            leaderElectionConfiguration,
+                                            configMapSharedWatcher,
+                                            watchExecutorService,
+                                            fatalErrorHandler));
+                } catch (Exception e) {
+                    throw new FlinkRuntimeException(
+                            "Could not initialize the default single leader election service.", e);
+                }
+            }
+
+            return multipleComponentLeaderElectionService;
+        }
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String componentId) {
+        return new DefaultLeaderRetrievalService(
+                new KubernetesMultipleComponentLeaderRetrievalDriverFactory(
+                        kubeClient,
+                        configMapSharedWatcher,
+                        watchExecutorService,
+                        getClusterConfigMap(),
+                        componentId));
+    }
+
+    @Override
+    protected CheckpointRecoveryFactory createCheckpointRecoveryFactory() {
+        return new KubernetesCheckpointRecoveryFactory(
+                kubeClient, configuration, ioExecutor, this::getJobSpecificConfigMap, lockIdentity);
+    }
+
+    private String getJobSpecificConfigMap(JobID jobID) {
+        return clusterId + NAME_SEPARATOR + jobID.toString() + NAME_SEPARATOR + "config-map";
+    }
+
+    @Override
+    protected JobGraphStore createJobGraphStore() throws Exception {
+        return KubernetesUtils.createJobGraphStore(
+                configuration, kubeClient, getClusterConfigMap(), lockIdentity);
+    }
+
+    private String getClusterConfigMap() {
+        return clusterId + NAME_SEPARATOR + "cluster-config-map";
+    }
+
+    @Override
+    protected RunningJobsRegistry createRunningJobsRegistry() {
+        return new KubernetesRunningJobsRegistry(kubeClient, getClusterConfigMap(), lockIdentity);
+    }
+
+    @Override
+    public void internalClose() throws Exception {
+        Exception exception = null;
+        try {
+            closeK8sServices();
+        } catch (Exception e) {
+            exception = e;
+        }
+
+        kubeClient.close();
+        ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, this.watchExecutorService);
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    private void closeK8sServices() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        configMapSharedWatcher.close();
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    public void internalCleanup() throws Exception {
+        Exception exception = null;
+        // in order to clean up, we first need to stop the services that rely on the config maps
+        try {
+            closeK8sServices();
+        } catch (Exception e) {
+            exception = e;
+        }
+
+        kubeClient
+                .deleteConfigMapsByLabels(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY))
+                .get();
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    public void internalCleanupJobData(JobID jobID) throws Exception {
+        kubeClient.deleteConfigMap(getJobSpecificConfigMap(jobID)).get();
+        // need to delete job specific leader address from leader config map
+    }
+
+    @Override
+    protected String getLeaderPathForResourceManager() {
+        return "resourcemanager";

Review comment:
       `KubernetesHaServices` will hopefully be removed in the next version. Hence, this will probably be work that can be spared.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -248,5 +229,24 @@ public void run(SourceContext<Integer> ctx) throws Exception {
         public void cancel() {
             running = false;
         }
+
+        @Override
+        public void snapshotState(FunctionSnapshotContext context) throws Exception {}
+
+        @Override
+        public void initializeState(FunctionInitializationContext context) throws Exception {
+            final ListState<Integer> hasExecutedBeforeState =
+                    context.getOperatorStateStore()
+                            .getUnionListState(hasExecutedBeforeStateDescriptor);
+
+            // if we have state, then we resume from a savepoint --> stop the execution then
+            if (hasExecutedBeforeState.get().iterator().hasNext()) {
+                running = false;
+            }
+
+            hasExecutedBeforeState.clear();
+            // mark this subtask as executed before
+            hasExecutedBeforeState.add(getRuntimeContext().getIndexOfThisSubtask());

Review comment:
       Yes, this is the idea of this state. Can rename it.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -248,5 +229,24 @@ public void run(SourceContext<Integer> ctx) throws Exception {
         public void cancel() {
             running = false;
         }
+
+        @Override
+        public void snapshotState(FunctionSnapshotContext context) throws Exception {}
+
+        @Override
+        public void initializeState(FunctionInitializationContext context) throws Exception {
+            final ListState<Integer> hasExecutedBeforeState =
+                    context.getOperatorStateStore()
+                            .getUnionListState(hasExecutedBeforeStateDescriptor);
+
+            // if we have state, then we resume from a savepoint --> stop the execution then
+            if (hasExecutedBeforeState.get().iterator().hasNext()) {
+                running = false;
+            }
+
+            hasExecutedBeforeState.clear();

Review comment:
       It is to have a clean state into which we insert the subtask index.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -248,5 +229,24 @@ public void run(SourceContext<Integer> ctx) throws Exception {
         public void cancel() {
             running = false;
         }
+
+        @Override
+        public void snapshotState(FunctionSnapshotContext context) throws Exception {}
+
+        @Override
+        public void initializeState(FunctionInitializationContext context) throws Exception {
+            final ListState<Integer> hasExecutedBeforeState =
+                    context.getOperatorStateStore()
+                            .getUnionListState(hasExecutedBeforeStateDescriptor);
+
+            // if we have state, then we resume from a savepoint --> stop the execution then
+            if (hasExecutedBeforeState.get().iterator().hasNext()) {
+                running = false;
+            }
+
+            hasExecutedBeforeState.clear();

Review comment:
       It is to have a clean state into which we insert the subtask index. It probably does not hurt to have it not but it also does not hurt to have it.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -141,20 +130,8 @@ public void testRecoverFromSavepoint() throws Exception {
         jobGraphWithSavepoint.setSavepointRestoreSettings(
                 SavepointRestoreSettings.forPath(savepoint2Path));
         clusterClient.submitJob(jobGraphWithSavepoint).get(TIMEOUT, TimeUnit.MILLISECONDS);
-        CommonTestUtils.waitUntilCondition(
-                () -> clusterClient.getJobStatus(jobId).get() == JobStatus.RUNNING,
-                Deadline.fromNow(Duration.ofMillis(TIMEOUT)),
-                1000);
 
-        // The savepoint 2 should be added to jobmanager leader ConfigMap
-        final String jobManagerConfigMapName = CLUSTER_ID + "-" + jobId + "-jobmanager-leader";
-        final Optional<KubernetesConfigMap> optional =
-                flinkKubeClient.getConfigMap(jobManagerConfigMapName);
-        assertThat(optional.isPresent(), is(true));
-        final String checkpointIdKey =
-                KubernetesCheckpointStoreUtil.INSTANCE.checkpointIDToName(2L);
-        assertThat(optional.get().getData().get(checkpointIdKey), is(notNullValue()));
-        assertThat(optional.get().getData().get(Constants.CHECKPOINT_COUNTER_KEY), is("3"));
+        assertThat(clusterClient.requestJobResult(jobId).join().isSuccess()).isTrue();

Review comment:
       I'll try to do this once all the fixup commits are squashed.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesTestFixture.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+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.kubernetes.kubeclient.resources.KubernetesLeaderElector;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+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.CompletionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+
+/** Test fixture for Kubernetes tests that sets up a mock {@link FlinkKubeClient}. */
+class KubernetesTestFixture {
+    private static final long TIMEOUT = 30L * 1000L;
+
+    private final String leaderConfigmapName;
+    private final String lockIdentity;
+
+    private final Configuration configuration;
+
+    private final Map<String, KubernetesConfigMap> configMapStore = new HashMap<>();
+
+    private final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+            configMapCallbackFutures = new ArrayList<>();
+
+    private final List<TestingFlinkKubeClient.MockKubernetesWatch> configMapWatches =
+            new ArrayList<>();
+
+    private final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture =
+            new CompletableFuture<>();
+    private final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+    private final CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
+            leaderCallbackHandlerFuture = new CompletableFuture<>();
+
+    private final FlinkKubeClient flinkKubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+
+    KubernetesTestFixture(String clusterId, String leaderConfigmapName, String lockIdentity) {
+        this.leaderConfigmapName = leaderConfigmapName;
+        this.lockIdentity = lockIdentity;
+        configuration = new Configuration();
+        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
+
+        flinkKubeClient = createFlinkKubeClient();
+        configMapSharedWatcher =
+                flinkKubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+    }
+
+    void close() {
+        configMapSharedWatcher.close();
+    }
+
+    FlinkKubeClient getFlinkKubeClient() {
+        return flinkKubeClient;
+    }
+
+    CompletableFuture<Void> getCloseKubeClientFuture() {
+        return closeKubeClientFuture;
+    }
+
+    CompletableFuture<Map<String, String>> getDeleteConfigMapByLabelsFuture() {
+        return deleteConfigMapByLabelsFuture;
+    }
+
+    KubernetesConfigMapSharedWatcher getConfigMapSharedWatcher() {
+        return configMapSharedWatcher;
+    }
+
+    Configuration getConfiguration() {
+        return configuration;
+    }
+
+    KubernetesConfigMap getLeaderConfigMap() {
+        final Optional<KubernetesConfigMap> configMapOpt =
+                flinkKubeClient.getConfigMap(leaderConfigmapName);
+        assertThat(configMapOpt.isPresent(), is(true));
+        return configMapOpt.get();
+    }
+
+    // Use the leader callback to manually grant leadership
+    void leaderCallbackGrantLeadership() throws Exception {
+        createLeaderConfigMap();
+        getLeaderCallback().isLeader();
+    }
+
+    FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> getLeaderElectionConfigMapCallback()
+            throws Exception {
+        assertThat(configMapCallbackFutures.size(), is(greaterThanOrEqualTo(1)));

Review comment:
       We probably are. I am also not super happy with the current state but I also didn't want to completely refactor the existing K8s tests. Hence, I would suggest to create a follow up task for cleaning this up.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesTestFixture.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+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.kubernetes.kubeclient.resources.KubernetesLeaderElector;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+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.CompletionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+
+/** Test fixture for Kubernetes tests that sets up a mock {@link FlinkKubeClient}. */
+class KubernetesTestFixture {
+    private static final long TIMEOUT = 30L * 1000L;
+
+    private final String leaderConfigmapName;
+    private final String lockIdentity;
+
+    private final Configuration configuration;
+
+    private final Map<String, KubernetesConfigMap> configMapStore = new HashMap<>();
+
+    private final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+            configMapCallbackFutures = new ArrayList<>();
+
+    private final List<TestingFlinkKubeClient.MockKubernetesWatch> configMapWatches =
+            new ArrayList<>();
+
+    private final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture =
+            new CompletableFuture<>();
+    private final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+    private final CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
+            leaderCallbackHandlerFuture = new CompletableFuture<>();
+
+    private final FlinkKubeClient flinkKubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+
+    KubernetesTestFixture(String clusterId, String leaderConfigmapName, String lockIdentity) {
+        this.leaderConfigmapName = leaderConfigmapName;
+        this.lockIdentity = lockIdentity;
+        configuration = new Configuration();
+        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
+
+        flinkKubeClient = createFlinkKubeClient();
+        configMapSharedWatcher =
+                flinkKubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+    }
+
+    void close() {
+        configMapSharedWatcher.close();
+    }
+
+    FlinkKubeClient getFlinkKubeClient() {
+        return flinkKubeClient;
+    }
+
+    CompletableFuture<Void> getCloseKubeClientFuture() {
+        return closeKubeClientFuture;
+    }
+
+    CompletableFuture<Map<String, String>> getDeleteConfigMapByLabelsFuture() {
+        return deleteConfigMapByLabelsFuture;
+    }
+
+    KubernetesConfigMapSharedWatcher getConfigMapSharedWatcher() {
+        return configMapSharedWatcher;
+    }
+
+    Configuration getConfiguration() {
+        return configuration;
+    }
+
+    KubernetesConfigMap getLeaderConfigMap() {
+        final Optional<KubernetesConfigMap> configMapOpt =
+                flinkKubeClient.getConfigMap(leaderConfigmapName);
+        assertThat(configMapOpt.isPresent(), is(true));
+        return configMapOpt.get();
+    }
+
+    // Use the leader callback to manually grant leadership
+    void leaderCallbackGrantLeadership() throws Exception {
+        createLeaderConfigMap();
+        getLeaderCallback().isLeader();
+    }
+
+    FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> getLeaderElectionConfigMapCallback()
+            throws Exception {
+        assertThat(configMapCallbackFutures.size(), is(greaterThanOrEqualTo(1)));
+        return configMapCallbackFutures.get(0).get(TIMEOUT, TimeUnit.MILLISECONDS);
+    }
+
+    FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> getLeaderRetrievalConfigMapCallback()
+            throws Exception {
+        assertThat(configMapCallbackFutures.size(), is(2));
+        return configMapCallbackFutures.get(1).get(TIMEOUT, TimeUnit.MILLISECONDS);
+    }
+
+    KubernetesLeaderElector.LeaderCallbackHandler getLeaderCallback() throws Exception {
+        return leaderCallbackHandlerFuture.get(TIMEOUT, TimeUnit.MILLISECONDS);
+    }
+
+    private FlinkKubeClient createFlinkKubeClient() {
+        return createFlinkKubeClientBuilder().build();
+    }
+
+    TestingFlinkKubeClient.Builder createFlinkKubeClientBuilder() {

Review comment:
       I rather would like to not do this because I don't see an immediate benefit. If this should be necessary in the future, then let's spend the time on it.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java
##########
@@ -53,6 +51,9 @@
  */
 public class KubernetesLeaderElectionAndRetrievalITCase extends TestLogger {

Review comment:
       I think this is a follow up task.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java
##########
@@ -53,6 +51,9 @@
  */
 public class KubernetesLeaderElectionAndRetrievalITCase extends TestLogger {

Review comment:
       I think this is a potential follow up task. Otherwise I fear that we are unnecessarily letting the scope of this PR grow.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();

Review comment:
       This is not a test but the actual signal to give leadership to the contender.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            getLeaderCallback().notLeader();
+                            leaderElectionListener.await(LeaderElectionEvent.NotLeaderEvent.class);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testPublishLeaderInformation() throws Exception {}
+    public void testPublishLeaderInformation() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();

Review comment:
       This is necessary to give leadership to the contender.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            getLeaderCallback().notLeader();
+                            leaderElectionListener.await(LeaderElectionEvent.NotLeaderEvent.class);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testPublishLeaderInformation() throws Exception {}
+    public void testPublishLeaderInformation() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            final LeaderInformation leaderInformation =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            final String componentId = "foobar";

Review comment:
       I'll change them.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            getLeaderCallback().notLeader();
+                            leaderElectionListener.await(LeaderElectionEvent.NotLeaderEvent.class);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testPublishLeaderInformation() throws Exception {}
+    public void testPublishLeaderInformation() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            final LeaderInformation leaderInformation =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            final String componentId = "foobar";
+
+                            final DefaultLeaderRetrievalService leaderRetrievalService =
+                                    new DefaultLeaderRetrievalService(
+                                            new KubernetesMultipleComponentLeaderRetrievalDriverFactory(
+                                                    getFlinkKubeClient(),
+                                                    getConfigMapSharedWatcher(),
+                                                    testExecutorExtension.getExecutor(),
+                                                    LEADER_CONFIGMAP_NAME,
+                                                    componentId));
+
+                            final TestingListener leaderRetrievalListener = new TestingListener();
+                            leaderRetrievalService.start(leaderRetrievalListener);
+
+                            leaderElectionDriver.publishLeaderInformation(
+                                    componentId, leaderInformation);
+
+                            notifyLeaderRetrievalWatchOnModifiedConfigMap();
+
+                            leaderRetrievalListener.waitForNewLeader(10_000L);
+                            assertThat(leaderRetrievalListener.getLeader())
+                                    .isEqualTo(leaderInformation);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testLeaderInformationChange() throws Exception {}
+    public void testLeaderInformationChangeNotifiesListener() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            final String componentA = "componentA";
+                            final LeaderInformation leaderInformationA =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            final String componentB = "componentB";
+                            final LeaderInformation leaderInformationB =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            leaderElectionDriver.publishLeaderInformation(
+                                    componentA, leaderInformationA);
+                            leaderElectionDriver.publishLeaderInformation(
+                                    componentB, leaderInformationB);
+
+                            notifyLeaderElectionWatchOnModifiedConfigMap();
+
+                            final LeaderElectionEvent.AllKnownLeaderInformationEvent
+                                    allKnownLeaderInformationEvent =
+                                            leaderElectionListener.await(
+                                                    LeaderElectionEvent
+                                                            .AllKnownLeaderInformationEvent.class);
+
+                            assertThat(
+                                            allKnownLeaderInformationEvent
+                                                    .getLeaderInformationWithComponentIds())
+                                    .contains(

Review comment:
       True. Will change it.

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
##########
@@ -204,6 +204,23 @@
                     .withDescription(
                             "The time before a JobManager after a fail over recovers the current jobs.");
 
+    /**
+     * Safety hatch to fallback to the old ha services implementations.
+     *
+     * <p>Ideally, we can remove this option together with the old implementations in the next
+     * release.
+     */
+    @Documentation.Section(Documentation.Sections.EXPERT_HIGH_AVAILABILITY)

Review comment:
       Yes. I think this is a good idea. I will create a ticket for this removal and add it in the comment.

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
##########
@@ -204,6 +204,23 @@
                     .withDescription(
                             "The time before a JobManager after a fail over recovers the current jobs.");
 
+    /**
+     * Safety hatch to fallback to the old ha services implementations.
+     *
+     * <p>Ideally, we can remove this option together with the old implementations in the next
+     * release.
+     */
+    @Documentation.Section(Documentation.Sections.EXPERT_HIGH_AVAILABILITY)

Review comment:
       https://issues.apache.org/jira/browse/FLINK-2580

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
##########
@@ -204,6 +204,23 @@
                     .withDescription(
                             "The time before a JobManager after a fail over recovers the current jobs.");
 
+    /**
+     * Safety hatch to fallback to the old ha services implementations.
+     *
+     * <p>Ideally, we can remove this option together with the old implementations in the next
+     * release.
+     */
+    @Documentation.Section(Documentation.Sections.EXPERT_HIGH_AVAILABILITY)

Review comment:
       https://issues.apache.org/jira/browse/FLINK-25806

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
##########
@@ -204,6 +204,23 @@
                     .withDescription(
                             "The time before a JobManager after a fail over recovers the current jobs.");
 
+    /**
+     * Safety hatch to fallback to the old ha services implementations.
+     *
+     * <p>Ideally, we can remove this option together with the old implementations in the next
+     * release.
+     */
+    @Documentation.Section(Documentation.Sections.EXPERT_HIGH_AVAILABILITY)
+    public static final ConfigOption<Boolean> USE_OLD_HA_SERVICES =
+            key("high-availability.use-old-ha-services")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "Use this option to disable the new HA service implementations for ZooKeeper and K8s. This is safety hatch in case that the new ha services are buggy.")

Review comment:
       Will correct it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
##########
@@ -82,6 +80,31 @@ public static HighAvailabilityServices createAvailableOrEmbeddedServices(
         }
     }
 
+    @Nonnull

Review comment:
       Yes. Will remove it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
##########
@@ -82,6 +80,31 @@ public static HighAvailabilityServices createAvailableOrEmbeddedServices(
         }
     }
 
+    @Nonnull
+    private static HighAvailabilityServices createZooKeeperHaServices(

Review comment:
       Because you can only activate the K8s ha services by configuring a factory. This has historical reasons.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -104,15 +102,6 @@ public void setup() throws Exception {
         savepointPath = temporaryFolder.newFolder("savepoints").getAbsolutePath();
     }
 
-    @After
-    public void teardown() throws Exception {
-        flinkKubeClient

Review comment:
       True. Will remove it.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java
##########
@@ -78,21 +84,46 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore(
             SharedStateRegistryFactory sharedStateRegistryFactory,
             Executor ioExecutor)
             throws Exception {
+        final String configMapName = getConfigMapNameFunction.apply(jobID);
+        KubernetesUtils.createConfigMapIfItDoesNotExist(kubeClient, configMapName, clusterId);
 
         return KubernetesUtils.createCompletedCheckpointStore(
                 configuration,
                 kubeClient,
                 executor,
-                getConfigMapNameFunction.apply(jobID),
+                configMapName,
                 lockIdentity,
                 maxNumberOfCheckpointsToRetain,
                 sharedStateRegistryFactory,
                 ioExecutor);
     }
 
     @Override
-    public CheckpointIDCounter createCheckpointIDCounter(JobID jobID) {
-        return new KubernetesCheckpointIDCounter(
-                kubeClient, getConfigMapNameFunction.apply(jobID), lockIdentity);
+    public CheckpointIDCounter createCheckpointIDCounter(JobID jobID) throws Exception {

Review comment:
       Then let's not 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull
+    private DefaultMultipleComponentLeaderElectionService
+            createDefaultMultiplexingLeaderElectionService(
+                    TestingMultipleComponentLeaderElectionDriver leaderElectionDriver)
+                    throws Exception {
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                new DefaultMultipleComponentLeaderElectionService(
+                        fatalErrorHandlerExtension.getTestingFatalErrorHandler(),
+                        "foobar",
+                        new TestingMultipleComponentLeaderElectionDriverFactory(
+                                leaderElectionDriver),
+                        Executors.newDirectExecutorService());
+        return leaderElectionService;
+    }
+
+    @Test
+    public void notLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+            leaderElectionDriver.revokeLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isFalse();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void unregisteredEventHandlersAreNotNotified() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            final String componentId = "foobar";
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    componentId, leaderElectionEventHandler);
+            leaderElectionService.unregisterLeaderElectionEventHandler(componentId);
+
+            leaderElectionDriver.grantLeadership();
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isFalse();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void newlyRegisteredEventHandlersAreInformedAboutLeadership() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            leaderElectionDriver.grantLeadership();
+
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    "foobar", leaderElectionEventHandler);
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isTrue();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void allKnownLeaderInformationCallsEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            leaderElectionDriver.grantLeadership();
+
+            final Collection<Component> knownLeaderInformation = createComponents(3);
+            final Collection<Component> unknownLeaderInformation = createComponents(2);

Review comment:
       I think I wanted to also register these components. Just not send their leader information in the `notifyAllKnownLeaderInformation`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull
+    private DefaultMultipleComponentLeaderElectionService
+            createDefaultMultiplexingLeaderElectionService(
+                    TestingMultipleComponentLeaderElectionDriver leaderElectionDriver)
+                    throws Exception {
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                new DefaultMultipleComponentLeaderElectionService(
+                        fatalErrorHandlerExtension.getTestingFatalErrorHandler(),
+                        "foobar",
+                        new TestingMultipleComponentLeaderElectionDriverFactory(
+                                leaderElectionDriver),
+                        Executors.newDirectExecutorService());
+        return leaderElectionService;
+    }
+
+    @Test
+    public void notLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+            leaderElectionDriver.revokeLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isFalse();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void unregisteredEventHandlersAreNotNotified() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            final String componentId = "foobar";
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    componentId, leaderElectionEventHandler);
+            leaderElectionService.unregisterLeaderElectionEventHandler(componentId);
+
+            leaderElectionDriver.grantLeadership();
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isFalse();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void newlyRegisteredEventHandlersAreInformedAboutLeadership() throws Exception {

Review comment:
       I am not sure whether this is really needed. I think it is fine to leave this undefined for the time being.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesTestFixture.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+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.kubernetes.kubeclient.resources.KubernetesLeaderElector;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+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.CompletionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+
+/** Test fixture for Kubernetes tests that sets up a mock {@link FlinkKubeClient}. */
+class KubernetesTestFixture {
+    private static final long TIMEOUT = 30L * 1000L;
+
+    private final String leaderConfigmapName;
+    private final String lockIdentity;
+
+    private final Configuration configuration;
+
+    private final Map<String, KubernetesConfigMap> configMapStore = new HashMap<>();
+
+    private final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+            configMapCallbackFutures = new ArrayList<>();
+
+    private final List<TestingFlinkKubeClient.MockKubernetesWatch> configMapWatches =
+            new ArrayList<>();
+
+    private final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture =
+            new CompletableFuture<>();
+    private final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+    private final CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
+            leaderCallbackHandlerFuture = new CompletableFuture<>();
+
+    private final FlinkKubeClient flinkKubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+
+    KubernetesTestFixture(String clusterId, String leaderConfigmapName, String lockIdentity) {
+        this.leaderConfigmapName = leaderConfigmapName;
+        this.lockIdentity = lockIdentity;
+        configuration = new Configuration();
+        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
+
+        flinkKubeClient = createFlinkKubeClient();
+        configMapSharedWatcher =
+                flinkKubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+    }
+
+    void close() {
+        configMapSharedWatcher.close();
+    }
+
+    FlinkKubeClient getFlinkKubeClient() {
+        return flinkKubeClient;
+    }
+
+    CompletableFuture<Void> getCloseKubeClientFuture() {
+        return closeKubeClientFuture;
+    }
+
+    CompletableFuture<Map<String, String>> getDeleteConfigMapByLabelsFuture() {
+        return deleteConfigMapByLabelsFuture;
+    }
+
+    KubernetesConfigMapSharedWatcher getConfigMapSharedWatcher() {
+        return configMapSharedWatcher;
+    }
+
+    Configuration getConfiguration() {
+        return configuration;
+    }
+
+    KubernetesConfigMap getLeaderConfigMap() {
+        final Optional<KubernetesConfigMap> configMapOpt =
+                flinkKubeClient.getConfigMap(leaderConfigmapName);
+        assertThat(configMapOpt.isPresent(), is(true));
+        return configMapOpt.get();
+    }
+
+    // Use the leader callback to manually grant leadership
+    void leaderCallbackGrantLeadership() throws Exception {
+        createLeaderConfigMap();
+        getLeaderCallback().isLeader();
+    }
+
+    FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> getLeaderElectionConfigMapCallback()
+            throws Exception {
+        assertThat(configMapCallbackFutures.size(), is(greaterThanOrEqualTo(1)));
+        return configMapCallbackFutures.get(0).get(TIMEOUT, TimeUnit.MILLISECONDS);
+    }
+
+    FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> getLeaderRetrievalConfigMapCallback()
+            throws Exception {
+        assertThat(configMapCallbackFutures.size(), is(2));
+        return configMapCallbackFutures.get(1).get(TIMEOUT, TimeUnit.MILLISECONDS);
+    }
+
+    KubernetesLeaderElector.LeaderCallbackHandler getLeaderCallback() throws Exception {
+        return leaderCallbackHandlerFuture.get(TIMEOUT, TimeUnit.MILLISECONDS);
+    }
+
+    private FlinkKubeClient createFlinkKubeClient() {
+        return createFlinkKubeClientBuilder().build();
+    }
+
+    TestingFlinkKubeClient.Builder createFlinkKubeClientBuilder() {

Review comment:
       I rather would like to not do this because I don't see an immediate benefit. If this should be necessary in the future, then let's spend the time on 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);
+
+            ExecutorUtils.gracefulShutdown(10L, TimeUnit.SECONDS, leadershipOperationExecutor);
+
+            Exception exception = null;
+            try {
+                multipleComponentLeaderElectionDriver.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public LeaderElectionDriverFactory createDriverFactory(String leaderName) {
+        return new MultipleComponentLeaderElectionDriverAdapterFactory(leaderName, this);
+    }
+
+    @Override
+    public void publishLeaderInformation(String leaderName, LeaderInformation leaderInformation) {
+        try {
+            multipleComponentLeaderElectionDriver.publishLeaderInformation(
+                    leaderName, leaderInformation);
+        } catch (Exception e) {
+            fatalErrorHandler.onFatalError(
+                    new FlinkException(
+                            String.format(
+                                    "Could not write leader information %s for leader %s.",
+                                    leaderInformation, leaderName),
+                            e));
+        }
+    }
+
+    @Override
+    public void registerLeaderElectionEventHandler(
+            String componentId, LeaderElectionEventHandler leaderElectionEventHandler) {
+
+        synchronized (lock) {
+            Preconditions.checkState(
+                    !leaderElectionEventHandlers.containsKey(componentId),
+                    "Do not support duplicate LeaderElectionEventHandler registration under %s",
+                    componentId);
+            leaderElectionEventHandlers.put(componentId, leaderElectionEventHandler);
+
+            if (currentLeaderSessionId != null) {
+                leadershipOperationExecutor.execute(
+                        () -> leaderElectionEventHandler.onGrantLeadership(currentLeaderSessionId));
+            }
+        }
+    }
+
+    @Override
+    public void unregisterLeaderElectionEventHandler(String componentId) throws Exception {
+        final LeaderElectionEventHandler unregisteredLeaderElectionEventHandler;
+        synchronized (lock) {
+            unregisteredLeaderElectionEventHandler =
+                    leaderElectionEventHandlers.remove(componentId);
+
+            if (unregisteredLeaderElectionEventHandler != null) {

Review comment:
       True. Will add 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                new ZooKeeperMultipleComponentLeaderElectionDriver(
+                        curatorFramework, "foobar", leaderElectionListener);
+        return leaderElectionDriver;
+    }
+
+    @Nonnull
+    private CuratorFrameworkWithUnhandledErrorListener startCuratorFramework() {
+        final Configuration configuration = new Configuration();
+        configuration.set(
+                HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperExtension.getConnectString());
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework =

Review comment:
       Will correct 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                new ZooKeeperMultipleComponentLeaderElectionDriver(
+                        curatorFramework, "foobar", leaderElectionListener);
+        return leaderElectionDriver;
+    }
+
+    @Nonnull
+    private CuratorFrameworkWithUnhandledErrorListener startCuratorFramework() {

Review comment:
       For compositional reasons we might also introduce a `CuratorFrameworkExtension`. I would leave this for a follow up task since it wouldn't change anything for this PR.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher;
+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.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.LeaderInformationWithComponentId;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriver;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+
+/** {@link MultipleComponentLeaderElectionDriver} for Kubernetes. */
+public class KubernetesMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(KubernetesMultipleComponentLeaderElectionDriver.class);
+
+    private final FlinkKubeClient kubeClient;
+
+    private final String configMapName;
+
+    private final String lockIdentity;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final KubernetesLeaderElector leaderElector;
+
+    // Labels will be used to clean up the ha related ConfigMaps.
+    private final Map<String, String> configMapLabels;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    private final KubernetesSharedWatcher.Watch kubernetesWatch;
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public KubernetesMultipleComponentLeaderElectionDriver(
+            KubernetesLeaderElectionConfiguration leaderElectionConfiguration,
+            FlinkKubeClient kubeClient,
+            Listener leaderElectionListener,
+            KubernetesConfigMapSharedWatcher configMapSharedWatcher,
+            Executor watchExecutor,
+            FatalErrorHandler fatalErrorHandler) {
+        this.kubeClient = kubeClient;
+        this.leaderElectionListener = leaderElectionListener;
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.configMapName = leaderElectionConfiguration.getConfigMapName();
+        this.lockIdentity = leaderElectionConfiguration.getLockIdentity();
+
+        this.leaderElector =
+                kubeClient.createLeaderElector(
+                        leaderElectionConfiguration, new LeaderCallbackHandlerImpl());
+
+        this.configMapLabels =
+                KubernetesUtils.getConfigMapLabels(
+                        leaderElectionConfiguration.getClusterId(),
+                        LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+        kubernetesWatch =
+                configMapSharedWatcher.watch(
+                        configMapName, new ConfigMapCallbackHandlerImpl(), watchExecutor);
+
+        leaderElector.run();

Review comment:
       I don't see a strong benefit other than complicating the interface. Therefore, I would like to not do it at this point in time.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",

Review comment:
       Let me try to remove the leader description.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java
##########
@@ -78,21 +84,46 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore(
             SharedStateRegistryFactory sharedStateRegistryFactory,
             Executor ioExecutor)
             throws Exception {
+        final String configMapName = getConfigMapNameFunction.apply(jobID);
+        KubernetesUtils.createConfigMapIfItDoesNotExist(kubeClient, configMapName, clusterId);
 
         return KubernetesUtils.createCompletedCheckpointStore(
                 configuration,
                 kubeClient,
                 executor,
-                getConfigMapNameFunction.apply(jobID),
+                configMapName,
                 lockIdentity,
                 maxNumberOfCheckpointsToRetain,
                 sharedStateRegistryFactory,
                 ioExecutor);
     }
 
     @Override
-    public CheckpointIDCounter createCheckpointIDCounter(JobID jobID) {
-        return new KubernetesCheckpointIDCounter(
-                kubeClient, getConfigMapNameFunction.apply(jobID), lockIdentity);
+    public CheckpointIDCounter createCheckpointIDCounter(JobID jobID) throws Exception {

Review comment:
       Then let's not 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -96,13 +106,17 @@
     /** The prefix of the completed checkpoint file. */
     public static final String HA_STORAGE_COMPLETED_CHECKPOINT = "completedCheckpoint";
 
-    private static final String RESOURCE_MANAGER_LEADER = "/resource_manager";
+    private static final String RESOURCE_MANAGER_LEADER = "resource_manager";
 
-    private static final String DISPATCHER_LEADER = "/dispatcher";
+    private static final String DISPATCHER_LEADER = "dispatcher";
 
     private static final String LEADER_NODE = "/leader";
 
-    private static final String REST_SERVER_LEADER = "/rest_server";
+    private static final String REST_SERVER_LEADER = "rest_server";
+
+    private static final String LEADER_LATCH_NODE = "/latch";

Review comment:
       Yes. 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -555,6 +668,10 @@ public static String generateZookeeperPath(String root, String child) {
         return result;
     }
 
+    public static String makeZooKeeperPath(String path) {

Review comment:
       True, I will change 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());

Review comment:
       Let me 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';
+
+    public static String encodeLeaderInformation(LeaderInformation leaderInformation) {
+        Preconditions.checkState(!leaderInformation.isEmpty());

Review comment:
       True. I will change 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull
+    private DefaultMultipleComponentLeaderElectionService
+            createDefaultMultiplexingLeaderElectionService(
+                    TestingMultipleComponentLeaderElectionDriver leaderElectionDriver)
+                    throws Exception {
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =

Review comment:
       Yes, I will update the code.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   * c7aaf594b9767994a95d9cba1c2f7e60d385002a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6934d6236ab71309c59fab4432aa3234fe916e12 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";

Review comment:
       Fair enough. I'll change 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
##########
@@ -204,6 +204,23 @@
                     .withDescription(
                             "The time before a JobManager after a fail over recovers the current jobs.");
 
+    /**
+     * Safety hatch to fallback to the old ha services implementations.
+     *
+     * <p>Ideally, we can remove this option together with the old implementations in the next
+     * release.
+     */
+    @Documentation.Section(Documentation.Sections.EXPERT_HIGH_AVAILABILITY)

Review comment:
       Yes. I think this is a good idea. I will create a ticket for this removal and add it in the comment.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher;
+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.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.LeaderInformationWithComponentId;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriver;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+
+/** {@link MultipleComponentLeaderElectionDriver} for Kubernetes. */
+public class KubernetesMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(KubernetesMultipleComponentLeaderElectionDriver.class);
+
+    private final FlinkKubeClient kubeClient;
+
+    private final String configMapName;
+
+    private final String lockIdentity;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final KubernetesLeaderElector leaderElector;
+
+    // Labels will be used to clean up the ha related ConfigMaps.
+    private final Map<String, String> configMapLabels;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    private final KubernetesSharedWatcher.Watch kubernetesWatch;
+
+    private AtomicBoolean running = new AtomicBoolean(true);

Review comment:
       True. 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = Preconditions.checkNotNull(curatorFramework);
+        this.leaderElectionListener = Preconditions.checkNotNull(leaderElectionListener);
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchPath());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_UPDATED:
+                                    Preconditions.checkNotNull(
+                                            event.getData(),
+                                            "The ZooKeeper event data must not be null.");
+                                    handleChangedLeaderInformation(event.getData());
+                                    break;
+                                case NODE_REMOVED:
+                                    Preconditions.checkNotNull(
+                                            event.getData(),
+                                            "The ZooKeeper event data must not be null.");
+                                    handleRemovedLeaderInformation(event.getData().getPath());
+                                    break;
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+
+        if (!leaderLatch.hasLeadership()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(
+                curatorFramework, ZooKeeperUtils.generateZookeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 no longer participates in the leader election.");
+                break;
+        }
+    }
+
+    @Override
+    public void isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (shouldHandleLeaderInformationEvent(childData.getPath())) {
+            final String leaderName = extractLeaderName(childData.getPath());
+
+            final LeaderInformation leaderInformation =
+                    tryReadingLeaderInformation(childData, leaderName);
+
+            leaderElectionListener.notifyLeaderInformationChange(leaderName, leaderInformation);
+        }
+    }
+
+    private String extractLeaderName(String path) {

Review comment:
       I think this is logic specific to the `ZKMCLEDriver` because it leverages the fact that the path is always `/leader_name/connection_info`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -402,6 +440,87 @@ public static ZooKeeperLeaderElectionDriverFactory createLeaderElectionDriverFac
         return new ZooKeeperLeaderElectionDriverFactory(client, path);
     }
 
+    public static void writeLeaderInformationToZooKeeper(
+            LeaderInformation leaderInformation,
+            CuratorFramework curatorFramework,
+            BooleanSupplier hasLeadershipCheck,
+            String connectionInformationPath)
+            throws Exception {
+        final byte[] data;
+
+        if (leaderInformation.isEmpty()) {
+            data = null;

Review comment:
       Do you mean a test where we write and read a `LeaderInformation.empty()`? This sounds like a good idea. If you meant a test where we use curator to write a `null` data, I think this not needed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {
+        // Maybe use a single service for leader retrieval
+        return ZooKeeperUtils.createLeaderRetrievalService(
+                leaderNamespacedCuratorFramework, leaderPath, configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        try {
+            super.internalClose();
+        } catch (Exception e) {
+            exception = ExceptionUtils.firstOrSuppressed(e, exception);
+        }
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        super.internalCleanupJobData(jobID);
+    }

Review comment:
       True. I will remove 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (running.get() && leaderLatch.hasLeadership() && isConnectionInfoNode(childData)) {
+
+            final String path = childData.getPath();
+            final String[] splits = path.split("/");
+
+            Preconditions.checkState(
+                    splits.length >= 2,
+                    String.format(
+                            "Expecting path consisting of <leader_name>/connection_info. Got path '%s'",
+                            path));
+            final String leaderName = splits[splits.length - 2];
+
+            final LeaderInformation leaderInformation =
+                    tryReadingLeaderInformation(childData, leaderName);
+
+            leaderElectionListener.notifyLeaderInformationChange(leaderName, leaderInformation);
+        }
+    }
+
+    private boolean isConnectionInfoNode(ChildData childData) {
+        return childData.getPath().endsWith(ZooKeeperUtils.CONNECTION_INFO_NODE);
+    }
+
+    private LeaderInformation tryReadingLeaderInformation(ChildData childData, String id) {
+        LeaderInformation leaderInformation;
+        try {
+            leaderInformation = ZooKeeperUtils.readLeaderInformation(childData.getData());
+
+            LOG.debug("Leader information for {} has changed to {}.", id, leaderInformation);
+        } catch (IOException | ClassNotFoundException e) {
+            LOG.debug(
+                    "Could not read leader information for {}. Rewriting the information.", id, e);
+            leaderInformation = LeaderInformation.empty();
+        }
+
+        return leaderInformation;
+    }
+
+    private static class ConnectionInfoNodeSelector implements TreeCacheSelector {

Review comment:
       Will do.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -96,13 +106,17 @@
     /** The prefix of the completed checkpoint file. */
     public static final String HA_STORAGE_COMPLETED_CHECKPOINT = "completedCheckpoint";
 
-    private static final String RESOURCE_MANAGER_LEADER = "/resource_manager";
+    private static final String RESOURCE_MANAGER_LEADER = "resource_manager";

Review comment:
       This has been addressed in a previous fixup.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';
+
+    public static String encodeLeaderInformation(LeaderInformation leaderInformation) {
+        Preconditions.checkState(!leaderInformation.isEmpty());
+        return leaderInformation.getLeaderSessionID().toString()
+                + LEADER_INFORMATION_SEPARATOR
+                + leaderInformation.getLeaderAddress();
+    }
+
+    public static LeaderInformation parseLeaderInformationSafely(String value) {
+        try {
+            return parseLeaderInformation(value);
+        } catch (Throwable throwable) {
+            LOG.debug("Could not parse value {} into LeaderInformation.", value, throwable);
+            return LeaderInformation.empty();
+        }
+    }
+
+    private static LeaderInformation parseLeaderInformation(String value) {
+        final int splitIndex = value.indexOf(LEADER_INFORMATION_SEPARATOR);
+
+        Preconditions.checkState(splitIndex >= 0, "Expecting '<session_id>,<leader_address>'");
+
+        final UUID leaderSessionId = UUID.fromString(value.substring(0, splitIndex));
+        final String leaderAddress = value.substring(splitIndex + 1);
+
+        return LeaderInformation.known(leaderSessionId, leaderAddress);
+    }
+
+    @Nonnull
+    public static String createSingleLeaderKey(String componentId) {
+        return LEADER_PREFIX + componentId;
+    }
+
+    public static boolean isSingleLeaderKey(String key) {
+        return key.startsWith(LEADER_PREFIX);
+    }
+
+    @Nonnull

Review comment:
       I don't think so. Will remove 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -248,5 +229,24 @@ public void run(SourceContext<Integer> ctx) throws Exception {
         public void cancel() {
             running = false;
         }
+
+        @Override
+        public void snapshotState(FunctionSnapshotContext context) throws Exception {}
+
+        @Override
+        public void initializeState(FunctionInitializationContext context) throws Exception {
+            final ListState<Integer> hasExecutedBeforeState =
+                    context.getOperatorStateStore()
+                            .getUnionListState(hasExecutedBeforeStateDescriptor);
+
+            // if we have state, then we resume from a savepoint --> stop the execution then
+            if (hasExecutedBeforeState.get().iterator().hasNext()) {
+                running = false;
+            }
+
+            hasExecutedBeforeState.clear();
+            // mark this subtask as executed before
+            hasExecutedBeforeState.add(getRuntimeContext().getIndexOfThisSubtask());

Review comment:
       Yes, this is the idea of this state. Can rename 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 972895e90d96516a2b8fa3722daf677e7759e266 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 972895e90d96516a2b8fa3722daf677e7759e266 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487) 
   * f8f21fd77139eee5f97f39195d31fcb67ae6d996 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f8f21fd77139eee5f97f39195d31fcb67ae6d996 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489) 
   * dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(

Review comment:
       Yes, the `fatalErrorHandler` is used for asynchronous exception that might occur. When doing synchronous call, I think it is better to directly throw the exception so that the caller can handle it appropriately.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionEvent.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.Collection;
+
+/** Leader election event. */
+public abstract class LeaderElectionEvent {
+    public boolean isIsLeaderEvent() {
+        return false;
+    }
+
+    public boolean isNotLeaderEvent() {
+        return false;
+    }
+
+    public boolean isLeaderInformationChangeEvent() {
+        return false;
+    }
+
+    public boolean isAllKnownLeaderInformationEvent() {
+        return false;
+    }
+
+    public IsLeaderEvent asIsLeaderEvent() {
+        return as(IsLeaderEvent.class);
+    }
+
+    public NotLeaderEvent asNotLeaderEvent() {
+        return as(NotLeaderEvent.class);
+    }
+
+    public LeaderInformationChangeEvent asLeaderInformationChangeEvent() {
+        return as(LeaderInformationChangeEvent.class);
+    }
+
+    public AllKnownLeaderInformationEvent asAllKnownLeaderInformationEvent() {
+        return as(AllKnownLeaderInformationEvent.class);
+    }

Review comment:
       True. I'll remove 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =

Review comment:
       True, will remove 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.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.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.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesConfigMapSharedWatcher;
+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.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+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.NAME_SEPARATOR;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Kubernetes HA services that use a single leader election service per JobManager. */
+public class KubernetesMultipleComponentLeaderElectionHaServices extends AbstractHaServices {
+
+    private final Object lock = new Object();
+
+    private final String clusterId;
+
+    private final FlinkKubeClient kubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+    private final ExecutorService watchExecutorService;
+
+    private final String lockIdentity;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private DefaultMultipleComponentLeaderElectionService multipleComponentLeaderElectionService =
+            null;
+
+    KubernetesMultipleComponentLeaderElectionHaServices(
+            FlinkKubeClient kubeClient,
+            Executor executor,
+            Configuration config,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler) {
+
+        super(config, executor, blobStoreService);
+        this.kubeClient = checkNotNull(kubeClient);
+        this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID));
+        this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+        this.configMapSharedWatcher =
+                this.kubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+        this.watchExecutorService =
+                Executors.newCachedThreadPool(
+                        new ExecutorThreadFactory("config-map-watch-handler"));
+
+        lockIdentity = UUID.randomUUID().toString();
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService =
+                getOrInitializeSingleLeaderElectionService();
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    private DefaultMultipleComponentLeaderElectionService
+            getOrInitializeSingleLeaderElectionService() {
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService == null) {
+                try {
+
+                    final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
+                            new KubernetesLeaderElectionConfiguration(
+                                    getClusterConfigMap(), lockIdentity, configuration);
+                    multipleComponentLeaderElectionService =
+                            new DefaultMultipleComponentLeaderElectionService(
+                                    fatalErrorHandler,
+                                    "Single leader election service",
+                                    new KubernetesMultipleComponentLeaderElectionDriverFactory(
+                                            kubeClient,
+                                            leaderElectionConfiguration,
+                                            configMapSharedWatcher,
+                                            watchExecutorService,
+                                            fatalErrorHandler));
+                } catch (Exception e) {
+                    throw new FlinkRuntimeException(
+                            "Could not initialize the default single leader election service.", e);
+                }
+            }
+
+            return multipleComponentLeaderElectionService;
+        }
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String componentId) {
+        return new DefaultLeaderRetrievalService(
+                new KubernetesMultipleComponentLeaderRetrievalDriverFactory(
+                        kubeClient,
+                        configMapSharedWatcher,
+                        watchExecutorService,
+                        getClusterConfigMap(),
+                        componentId));
+    }
+
+    @Override
+    protected CheckpointRecoveryFactory createCheckpointRecoveryFactory() {
+        return new KubernetesCheckpointRecoveryFactory(
+                kubeClient, configuration, ioExecutor, this::getJobSpecificConfigMap, lockIdentity);
+    }
+
+    private String getJobSpecificConfigMap(JobID jobID) {
+        return clusterId + NAME_SEPARATOR + jobID.toString() + NAME_SEPARATOR + "config-map";
+    }
+
+    @Override
+    protected JobGraphStore createJobGraphStore() throws Exception {
+        return KubernetesUtils.createJobGraphStore(
+                configuration, kubeClient, getClusterConfigMap(), lockIdentity);
+    }
+
+    private String getClusterConfigMap() {
+        return clusterId + NAME_SEPARATOR + "cluster-config-map";
+    }
+
+    @Override
+    protected RunningJobsRegistry createRunningJobsRegistry() {
+        return new KubernetesRunningJobsRegistry(kubeClient, getClusterConfigMap(), lockIdentity);
+    }
+
+    @Override
+    public void internalClose() throws Exception {
+        Exception exception = null;
+        try {
+            closeK8sServices();
+        } catch (Exception e) {
+            exception = e;
+        }
+
+        kubeClient.close();
+        ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, this.watchExecutorService);
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    private void closeK8sServices() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        configMapSharedWatcher.close();
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    public void internalCleanup() throws Exception {
+        Exception exception = null;
+        // in order to clean up, we first need to stop the services that rely on the config maps
+        try {
+            closeK8sServices();
+        } catch (Exception e) {
+            exception = e;
+        }
+
+        kubeClient
+                .deleteConfigMapsByLabels(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY))
+                .get();
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    public void internalCleanupJobData(JobID jobID) throws Exception {
+        kubeClient.deleteConfigMap(getJobSpecificConfigMap(jobID)).get();
+        // need to delete job specific leader address from leader config map
+    }
+
+    @Override
+    protected String getLeaderPathForResourceManager() {
+        return "resourcemanager";

Review comment:
       `KubernetesHaServices` will hopefully be removed in the next version. Hence, this will probably be work that can be spared.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);

Review comment:
       I think I wanted to put `this.getClass().getSimpleName()` there. 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull

Review comment:
       Yes, I will correct 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   * c7aaf594b9767994a95d9cba1c2f7e60d385002a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a605d8da083d456fbbae4c5adaaea270263b2f38 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473) 
   * 972895e90d96516a2b8fa3722daf677e7759e266 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull

Review comment:
       Will remove 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -402,6 +440,87 @@ public static ZooKeeperLeaderElectionDriverFactory createLeaderElectionDriverFac
         return new ZooKeeperLeaderElectionDriverFactory(client, path);
     }
 
+    public static void writeLeaderInformationToZooKeeper(
+            LeaderInformation leaderInformation,
+            CuratorFramework curatorFramework,
+            BooleanSupplier hasLeadershipCheck,
+            String connectionInformationPath)
+            throws Exception {
+        final byte[] data;
+
+        if (leaderInformation.isEmpty()) {
+            data = null;

Review comment:
       Do you mean a test where we write and read a `LeaderInformation.empty()`? This sounds like a good idea. If you meant a test where we use curator to write a `null` data, I think this not needed given the other test.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (running.get() && leaderLatch.hasLeadership() && isConnectionInfoNode(childData)) {
+
+            final String path = childData.getPath();
+            final String[] splits = path.split("/");

Review comment:
       Agreed for the splitting. The leaderName retrieval is specific to the driver and the used ha services. That's why I would like to keep it 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {

Review comment:
       Yes, I think we should also write `LeaderInformation.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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
##########
@@ -82,6 +80,31 @@ public static HighAvailabilityServices createAvailableOrEmbeddedServices(
         }
     }
 
+    @Nonnull
+    private static HighAvailabilityServices createZooKeeperHaServices(

Review comment:
       Because you can only activate the K8s ha services by configuring a factory. This has historical reasons.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+
+import javax.annotation.Nonnull;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Abstract ZooKeeper based HA services. */
+public abstract class AbstractZooKeeperHaServices extends AbstractHaServices {
+    /** The curator resource to use. */
+    private final CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper;
+
+    public AbstractZooKeeperHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Executor executor,
+            Configuration configuration,
+            BlobStoreService blobStoreService) {
+        super(configuration, executor, blobStoreService);
+        this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper);
+    }
+
+    protected final CuratorFramework getCuratorFramework() {
+        return curatorFrameworkWrapper.asCuratorFramework();
+    }
+
+    @Override
+    public CheckpointRecoveryFactory createCheckpointRecoveryFactory() throws Exception {
+        return new ZooKeeperCheckpointRecoveryFactory(
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        curatorFrameworkWrapper.asCuratorFramework(), ZooKeeperUtils.getJobsPath()),
+                configuration,
+                ioExecutor);
+    }
+
+    @Override
+    public JobGraphStore createJobGraphStore() throws Exception {
+        return ZooKeeperUtils.createJobGraphs(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    public RunningJobsRegistry createRunningJobsRegistry() {
+        return new ZooKeeperRunningJobsRegistry(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        curatorFrameworkWrapper.close();
+    }
+
+    @Override
+    protected void internalCleanup() throws Exception {
+        cleanupZooKeeperPaths();
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        deleteZNode(ZooKeeperUtils.getLeaderPathForJob(jobID));
+    }
+
+    /** Cleans up leftover ZooKeeper paths. */
+    private void cleanupZooKeeperPaths() throws Exception {
+        deleteOwnedZNode();
+        tryDeleteEmptyParentZNodes();
+    }
+
+    private void deleteOwnedZNode() throws Exception {
+        deleteZNode("/");
+    }
+
+    protected void deleteZNode(String path) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFrameworkWrapper.asCuratorFramework(), path);
+    }
+
+    /**
+     * Tries to delete empty parent znodes.
+     *
+     * <p>IMPORTANT: This method can be removed once all supported ZooKeeper versions support the
+     * container {@link org.apache.zookeeper.CreateMode}.
+     *
+     * @throws Exception if the deletion fails for other reason than {@link
+     *     KeeperException.NotEmptyException}
+     */
+    private void tryDeleteEmptyParentZNodes() throws Exception {
+        // try to delete the parent znodes if they are empty
+        String remainingPath =
+                getParentPath(
+                        getNormalizedPath(
+                                curatorFrameworkWrapper.asCuratorFramework().getNamespace()));
+        final CuratorFramework nonNamespaceClient =
+                curatorFrameworkWrapper.asCuratorFramework().usingNamespace(null);
+
+        while (!isRootPath(remainingPath)) {
+            try {
+                nonNamespaceClient.delete().forPath(remainingPath);
+            } catch (KeeperException.NotEmptyException ignored) {
+                // We can only delete empty znodes
+                break;
+            }
+
+            remainingPath = getParentPath(remainingPath);
+        }
+    }
+
+    private static boolean isRootPath(String remainingPath) {
+        return ZKPaths.PATH_SEPARATOR.equals(remainingPath);
+    }
+
+    @Nonnull

Review comment:
       Will remove it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+
+import javax.annotation.Nonnull;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Abstract ZooKeeper based HA services. */
+public abstract class AbstractZooKeeperHaServices extends AbstractHaServices {
+    /** The curator resource to use. */
+    private final CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper;
+
+    public AbstractZooKeeperHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Executor executor,
+            Configuration configuration,
+            BlobStoreService blobStoreService) {
+        super(configuration, executor, blobStoreService);
+        this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper);
+    }
+
+    protected final CuratorFramework getCuratorFramework() {
+        return curatorFrameworkWrapper.asCuratorFramework();
+    }
+
+    @Override
+    public CheckpointRecoveryFactory createCheckpointRecoveryFactory() throws Exception {
+        return new ZooKeeperCheckpointRecoveryFactory(
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        curatorFrameworkWrapper.asCuratorFramework(), ZooKeeperUtils.getJobsPath()),
+                configuration,
+                ioExecutor);
+    }
+
+    @Override
+    public JobGraphStore createJobGraphStore() throws Exception {
+        return ZooKeeperUtils.createJobGraphs(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    public RunningJobsRegistry createRunningJobsRegistry() {
+        return new ZooKeeperRunningJobsRegistry(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        curatorFrameworkWrapper.close();
+    }
+
+    @Override
+    protected void internalCleanup() throws Exception {
+        cleanupZooKeeperPaths();
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        deleteZNode(ZooKeeperUtils.getLeaderPathForJob(jobID));
+    }
+
+    /** Cleans up leftover ZooKeeper paths. */
+    private void cleanupZooKeeperPaths() throws Exception {
+        deleteOwnedZNode();
+        tryDeleteEmptyParentZNodes();
+    }
+
+    private void deleteOwnedZNode() throws Exception {
+        deleteZNode("/");
+    }
+
+    protected void deleteZNode(String path) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFrameworkWrapper.asCuratorFramework(), path);
+    }
+
+    /**
+     * Tries to delete empty parent znodes.
+     *
+     * <p>IMPORTANT: This method can be removed once all supported ZooKeeper versions support the
+     * container {@link org.apache.zookeeper.CreateMode}.
+     *
+     * @throws Exception if the deletion fails for other reason than {@link
+     *     KeeperException.NotEmptyException}
+     */
+    private void tryDeleteEmptyParentZNodes() throws Exception {
+        // try to delete the parent znodes if they are empty
+        String remainingPath =
+                getParentPath(
+                        getNormalizedPath(
+                                curatorFrameworkWrapper.asCuratorFramework().getNamespace()));
+        final CuratorFramework nonNamespaceClient =
+                curatorFrameworkWrapper.asCuratorFramework().usingNamespace(null);
+
+        while (!isRootPath(remainingPath)) {
+            try {
+                nonNamespaceClient.delete().forPath(remainingPath);
+            } catch (KeeperException.NotEmptyException ignored) {
+                // We can only delete empty znodes
+                break;
+            }
+
+            remainingPath = getParentPath(remainingPath);
+        }
+    }
+
+    private static boolean isRootPath(String remainingPath) {
+        return ZKPaths.PATH_SEPARATOR.equals(remainingPath);
+    }
+
+    @Nonnull
+    private static String getNormalizedPath(String path) {
+        return ZKPaths.makePath(path, "");
+    }
+
+    @Nonnull

Review comment:
       Will remove 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {

Review comment:
       I'll add `testPublishEmptyLeaderInformation` and `testNonLeaderCannotPublishLeaderInformation`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';
+
+    public static String encodeLeaderInformation(LeaderInformation leaderInformation) {
+        Preconditions.checkState(!leaderInformation.isEmpty());
+        return leaderInformation.getLeaderSessionID().toString()
+                + LEADER_INFORMATION_SEPARATOR
+                + leaderInformation.getLeaderAddress();
+    }
+
+    public static LeaderInformation parseLeaderInformationSafely(String value) {

Review comment:
       I think this does not make a huge difference. But can do.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann closed pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f8f21fd77139eee5f97f39195d31fcb67ae6d996 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();

Review comment:
       What would be the benefit? Unless there is something, I'd like to keep it as is.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));

Review comment:
       Will change 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';

Review comment:
       Let's stick to the existing conventions. I'll move it up.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6934d6236ab71309c59fab4432aa3234fe916e12 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056) 
   * 00e0b2324b8c72cf98105f985d40c006a43d6ddf Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {
+        // Maybe use a single service for leader retrieval
+        return ZooKeeperUtils.createLeaderRetrievalService(
+                leaderNamespacedCuratorFramework, leaderPath, configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        try {
+            super.internalClose();
+        } catch (Exception e) {
+            exception = ExceptionUtils.firstOrSuppressed(e, exception);
+        }
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        super.internalCleanupJobData(jobID);
+    }
+
+    @Override
+    protected String getLeaderPathForResourceManager() {
+        return ZooKeeperUtils.getResourceManagerNode();
+    }
+
+    @Override
+    protected String getLeaderPathForDispatcher() {
+        return ZooKeeperUtils.getDispatcherNode();
+    }
+
+    @Override
+    protected String getLeaderPathForJobManager(JobID jobID) {
+        return jobID.toString();
+    }
+
+    @Override
+    protected String getLeaderPathForRestServer() {
+        return ZooKeeperUtils.getRestServerNode();
+    }

Review comment:
       I think you are right that there is a difference between the method name and what is returned. In this case I would suggest to change the naming of the methods to `getLeaderNameXYZ` instead of `getLeaderPathXZY`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);
+
+            ExecutorUtils.gracefulShutdown(10L, TimeUnit.SECONDS, leadershipOperationExecutor);
+
+            Exception exception = null;
+            try {
+                multipleComponentLeaderElectionDriver.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            ExceptionUtils.tryRethrowException(exception);

Review comment:
       No, you are right. I 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(

Review comment:
       Indeed. 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionDriverFactory.java
##########
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 {@link MultipleComponentLeaderElectionDriver}. */
+public interface MultipleComponentLeaderElectionDriverFactory {
+
+    /**
+     * Creates a {@link MultipleComponentLeaderElectionDriver} for the given leader contender
+     * description. Moreover, it registers the given leader election listener with the service.

Review comment:
       The `MultipleComponentLeaderElectionDriverAdapter` is an implementation of the `LeaderElectionDriver` and not the `MultipleComponentLeaderElectionDriver`. Therefore, I think the comment makes sense. It refers to the fact that the given listener will be registered with the created MCLElectionDriver.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingMultipleComponentLeaderElectionDriver.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.runtime.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.util.Optional;
+
+/** Testing implementation of {@link MultipleComponentLeaderElectionDriver}. */
+public class TestingMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private final BiConsumerWithException<String, LeaderInformation, Exception>
+            publishLeaderInformationConsumer;
+    private final ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer;
+    private boolean hasLeadership;
+
+    private Optional<Listener> listener;
+
+    private TestingMultipleComponentLeaderElectionDriver(
+            BiConsumerWithException<String, LeaderInformation, Exception>
+                    publishLeaderInformationConsumer,
+            ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer) {
+        this.publishLeaderInformationConsumer = publishLeaderInformationConsumer;
+        this.deleteLeaderInformationConsumer = deleteLeaderInformationConsumer;
+        hasLeadership = false;
+        listener = Optional.empty();
+    }
+
+    public void grantLeadership() {
+        if (!hasLeadership) {
+            hasLeadership = true;
+            listener.ifPresent(Listener::isLeader);
+        }
+    }
+
+    public void revokeLeadership() {
+        if (hasLeadership) {
+            hasLeadership = false;
+            listener.ifPresent(Listener::notLeader);
+        }
+    }
+
+    public void setListener(Listener listener) {
+        Preconditions.checkState(!this.listener.isPresent(), "Can only set a single listener.");
+        this.listener = Optional.of(listener);
+    }
+
+    @Override
+    public void close() throws Exception {}
+
+    @Override
+    public boolean hasLeadership() {
+        return hasLeadership;
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        publishLeaderInformationConsumer.accept(componentId, leaderInformation);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String componentId) throws Exception {
+        deleteLeaderInformationConsumer.accept(componentId);
+    }
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    public static final class Builder {
+        private BiConsumerWithException<String, LeaderInformation, Exception>
+                publishLeaderInformationConsumer = (ignoredA, ignoredB) -> {};
+        private ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer =
+                (ignored) -> {};

Review comment:
       True. 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';

Review comment:
       Yes, this is a good idea.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();

Review comment:
       This is not a test but the actual signal to give leadership to the contender.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -104,15 +102,6 @@ public void setup() throws Exception {
         savepointPath = temporaryFolder.newFolder("savepoints").getAbsolutePath();
     }
 
-    @After
-    public void teardown() throws Exception {
-        flinkKubeClient

Review comment:
       True. Will remove 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f8f21fd77139eee5f97f39195d31fcb67ae6d996 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489) 
   * dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503) 
   * a3c818cb7a4341243bbf7f584d327a1f51d355d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155) 
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] XComp commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = Preconditions.checkNotNull(curatorFramework);
+        this.leaderElectionListener = Preconditions.checkNotNull(leaderElectionListener);
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchPath());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_UPDATED:
+                                    Preconditions.checkNotNull(
+                                            event.getData(),
+                                            "The ZooKeeper event data must not be null.");
+                                    handleChangedLeaderInformation(event.getData());
+                                    break;
+                                case NODE_REMOVED:
+                                    Preconditions.checkNotNull(
+                                            event.getData(),
+                                            "The ZooKeeper event data must not be null.");
+                                    handleRemovedLeaderInformation(event.getData().getPath());
+                                    break;
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+
+        if (!leaderLatch.hasLeadership()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(
+                curatorFramework, ZooKeeperUtils.generateZookeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 no longer participates in the leader election.");
+                break;
+        }
+    }
+
+    @Override
+    public void isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (shouldHandleLeaderInformationEvent(childData.getPath())) {
+            final String leaderName = extractLeaderName(childData.getPath());
+
+            final LeaderInformation leaderInformation =
+                    tryReadingLeaderInformation(childData, leaderName);
+
+            leaderElectionListener.notifyLeaderInformationChange(leaderName, leaderInformation);
+        }
+    }
+
+    private String extractLeaderName(String path) {

Review comment:
       nit: Feels like it should go into `ZooKeeperUtils` as well.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    private void handleStateChange(ConnectionState newState) {

Review comment:
       makes sense...

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -402,6 +440,87 @@ public static ZooKeeperLeaderElectionDriverFactory createLeaderElectionDriverFac
         return new ZooKeeperLeaderElectionDriverFactory(client, path);
     }
 
+    public static void writeLeaderInformationToZooKeeper(
+            LeaderInformation leaderInformation,
+            CuratorFramework curatorFramework,
+            BooleanSupplier hasLeadershipCheck,
+            String connectionInformationPath)
+            throws Exception {
+        final byte[] data;
+
+        if (leaderInformation.isEmpty()) {
+            data = null;

Review comment:
       can we add a test case to `ZooKeeperUtilsTest` for writing data (including `null`) into the path? I did some digging into the Curator4 code tot check: It looks like it is. But it's hard to find information here. The [JavaDoc](https://www.javadoc.io/doc/org.apache.curator/curator-framework/4.0.0/org/apache/curator/framework/api/PathAndBytesable.html) is also not helpful. WDYT? Is it worth the effort?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();

Review comment:
       True, I misread the code here 👍 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    @GuardedBy("lock")
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = Preconditions.checkNotNull(fatalErrorHandler);
+
+        this.leadershipOperationExecutor = Preconditions.checkNotNull(leadershipOperationExecutor);
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                multipleComponentLeaderElectionDriverFactory,
+                Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(String.format("leadershipOperationExecutor"))));

Review comment:
       ```suggestion
                           new ExecutorThreadFactory("leadershipOperationExecutor")));
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher;
+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.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.LeaderInformationWithComponentId;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriver;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+
+/** {@link MultipleComponentLeaderElectionDriver} for Kubernetes. */
+public class KubernetesMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(KubernetesMultipleComponentLeaderElectionDriver.class);
+
+    private final FlinkKubeClient kubeClient;
+
+    private final String configMapName;
+
+    private final String lockIdentity;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final KubernetesLeaderElector leaderElector;
+
+    // Labels will be used to clean up the ha related ConfigMaps.
+    private final Map<String, String> configMapLabels;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    private final KubernetesSharedWatcher.Watch kubernetesWatch;
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public KubernetesMultipleComponentLeaderElectionDriver(
+            KubernetesLeaderElectionConfiguration leaderElectionConfiguration,
+            FlinkKubeClient kubeClient,
+            Listener leaderElectionListener,
+            KubernetesConfigMapSharedWatcher configMapSharedWatcher,
+            Executor watchExecutor,
+            FatalErrorHandler fatalErrorHandler) {
+        this.kubeClient = kubeClient;
+        this.leaderElectionListener = leaderElectionListener;
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.configMapName = leaderElectionConfiguration.getConfigMapName();
+        this.lockIdentity = leaderElectionConfiguration.getLockIdentity();
+
+        this.leaderElector =
+                kubeClient.createLeaderElector(
+                        leaderElectionConfiguration, new LeaderCallbackHandlerImpl());
+
+        this.configMapLabels =
+                KubernetesUtils.getConfigMapLabels(
+                        leaderElectionConfiguration.getClusterId(),
+                        LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+        kubernetesWatch =
+                configMapSharedWatcher.watch(
+                        configMapName, new ConfigMapCallbackHandlerImpl(), watchExecutor);
+
+        leaderElector.run();

Review comment:
       IMHO, it would improve the code readability: At least, I wouldn't expect the actual execution being triggered (similar to `Thread`). But anyway, that's no big issue since it's a pattern that is also used in other `LeaderElectionDriver` implementations.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionDriverFactory.java
##########
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 {@link MultipleComponentLeaderElectionDriver}. */
+public interface MultipleComponentLeaderElectionDriverFactory {
+
+    /**
+     * Creates a {@link MultipleComponentLeaderElectionDriver} for the given leader contender
+     * description. Moreover, it registers the given leader election listener with the service.

Review comment:
       You're right. The corresponding `MultipleComponentLeaderElectionDriver` implementations register the listener with the corresponding backend in the driver's constructor. I didn't pay enough attention here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = Preconditions.checkNotNull(curatorFramework);
+        this.leaderElectionListener = Preconditions.checkNotNull(leaderElectionListener);
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchPath());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_UPDATED:
+                                    Preconditions.checkNotNull(
+                                            event.getData(),
+                                            "The ZooKeeper event data must not be null.");
+                                    handleChangedLeaderInformation(event.getData());
+                                    break;
+                                case NODE_REMOVED:
+                                    Preconditions.checkNotNull(
+                                            event.getData(),
+                                            "The ZooKeeper event data must not be null.");
+                                    handleRemovedLeaderInformation(event.getData().getPath());
+                                    break;
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+
+        if (!leaderLatch.hasLeadership()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(
+                curatorFramework, ZooKeeperUtils.generateZookeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 no longer participates in the leader election.");
+                break;
+        }
+    }
+
+    @Override
+    public void isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (shouldHandleLeaderInformationEvent(childData.getPath())) {
+            final String leaderName = extractLeaderName(childData.getPath());
+
+            final LeaderInformation leaderInformation =
+                    tryReadingLeaderInformation(childData, leaderName);
+
+            leaderElectionListener.notifyLeaderInformationChange(leaderName, leaderInformation);
+        }
+    }
+
+    private String extractLeaderName(String path) {
+        final String[] splits = ZooKeeperUtils.splitZooKeeperPath(path);
+
+        Preconditions.checkState(
+                splits.length >= 2,
+                String.format(
+                        "Expecting path consisting of /<leader_name>/connection_info. Got path '%s'",
+                        path));
+
+        return splits[splits.length - 2];
+    }
+
+    private void handleRemovedLeaderInformation(String removedNodePath) {
+        if (shouldHandleLeaderInformationEvent(removedNodePath)) {
+            final String leaderName = extractLeaderName(removedNodePath);
+
+            leaderElectionListener.notifyLeaderInformationChange(
+                    leaderName, LeaderInformation.empty());
+        }
+    }
+
+    private boolean shouldHandleLeaderInformationEvent(String removedNode) {
+        return running.get() && leaderLatch.hasLeadership() && isConnectionInfoNode(removedNode);

Review comment:
       ```suggestion
       private boolean shouldHandleLeaderInformationEvent(String path) {
           return running.get() && leaderLatch.hasLeadership() && isConnectionInfoNode(path);
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();

Review comment:
       As mentioned in another comment: I think that we gain a clearer separation of concerns here (constructor does the initialization and starting the resources is considered business logic; similar to how it's handled in `java.lang.Thread`). But let's keep it like that. That's probably out-of-scope for this PR.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] XComp commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -402,6 +440,87 @@ public static ZooKeeperLeaderElectionDriverFactory createLeaderElectionDriverFac
         return new ZooKeeperLeaderElectionDriverFactory(client, path);
     }
 
+    public static void writeLeaderInformationToZooKeeper(
+            LeaderInformation leaderInformation,
+            CuratorFramework curatorFramework,
+            BooleanSupplier hasLeadershipCheck,
+            String connectionInformationPath)
+            throws Exception {
+        final byte[] data;
+
+        if (leaderInformation.isEmpty()) {
+            data = null;

Review comment:
       Yup, agreed 👍 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 00e0b2324b8c72cf98105f985d40c006a43d6ddf Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6934d6236ab71309c59fab4432aa3234fe916e12 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056) 
   * 00e0b2324b8c72cf98105f985d40c006a43d6ddf UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 972895e90d96516a2b8fa3722daf677e7759e266 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487) 
   * f8f21fd77139eee5f97f39195d31fcb67ae6d996 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;

Review comment:
       I am a bit torn here. The general Flink guideline says that arguments that are not annotated with `@Nullable` can be assumed to be non-null. Hence, a null check should not strictly be necessary. On the other hand, failing fast in case of a NPE is always a good idea.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {

Review comment:
       Yes, I think you are right that this should be a `checkState` for the `event.getData() != null`. Moreover, I think that I am handling the `NODE_REMOVED` event wrongly. In this case, the leader information should be marked as `LeaderInformation.empty` because the event will be created with the old node data. I will correct the code.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -248,5 +229,24 @@ public void run(SourceContext<Integer> ctx) throws Exception {
         public void cancel() {
             running = false;
         }
+
+        @Override
+        public void snapshotState(FunctionSnapshotContext context) throws Exception {}
+
+        @Override
+        public void initializeState(FunctionInitializationContext context) throws Exception {
+            final ListState<Integer> hasExecutedBeforeState =
+                    context.getOperatorStateStore()
+                            .getUnionListState(hasExecutedBeforeStateDescriptor);
+
+            // if we have state, then we resume from a savepoint --> stop the execution then
+            if (hasExecutedBeforeState.get().iterator().hasNext()) {
+                running = false;
+            }
+
+            hasExecutedBeforeState.clear();

Review comment:
       It is to have a clean state into which we insert the subtask index. It probably does not hurt to have it not but it also does not hurt to have 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   Thanks for the review @XComp. I've resolved most of your comments. Those that I haven't resolved, I added an explanation for why not. Please take another look at this PR so that we can finish it swiftly :-)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503) 
   * a3c818cb7a4341243bbf7f584d327a1f51d355d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155) 
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   Thanks for the review @XComp. I've addressed your comments. Moreover, I squashed all fixup commits and rebased onto the latest master. I hope that the PR is now ready to be merged.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingMultipleComponentLeaderElectionDriver.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.runtime.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.util.Optional;
+
+/** Testing implementation of {@link MultipleComponentLeaderElectionDriver}. */
+public class TestingMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private final BiConsumerWithException<String, LeaderInformation, Exception>
+            publishLeaderInformationConsumer;
+    private final ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer;
+    private boolean hasLeadership;
+
+    private Optional<Listener> listener;

Review comment:
       I think this is because of your inspection settings.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {

Review comment:
       Yes, I think what we would save by moving it to the `AbstractZooKeeperHaServices` does not offset the potential costs that the leader election and retrieval service are not close together which makes it easier to keep them in sync.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6934d6236ab71309c59fab4432aa3234fe916e12 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Collection;
+
+/**
+ * A leader election driver that allows to write {@link LeaderInformation} for multiple components.
+ */
+public interface MultipleComponentLeaderElectionDriver {
+
+    /**
+     * Closes the driver.
+     *
+     * @throws Exception if closing this driver fails
+     */
+    void close() throws Exception;
+
+    /**
+     * Returns whether the driver has currently leadership.
+     *
+     * @return {@code true} if the driver has leadership, otherwise {@code false}
+     */
+    boolean hasLeadership();
+
+    /**
+     * Publishes the leader information for the given component.
+     *
+     * @param componentId identifying the component for which to publish the leader information
+     * @param leaderInformation leader information of the respective component
+     * @throws Exception if publishing fails
+     */
+    void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception;
+
+    /**
+     * Deletes the leader information for the given component.
+     *
+     * @param componentId identifying the component for which to delete the leader information
+     * @throws Exception if deleting fails
+     */
+    void deleteLeaderInformation(String componentId) throws Exception;
+
+    /**
+     * Listener interface for state changes of the {@link MultipleComponentLeaderElectionDriver}.
+     */
+    interface Listener {
+
+        /** Callback that is called once the driver obtains the leadership. */
+        void isLeader();
+
+        /** Callback that is called once the driver loses the leadership. */
+        void notLeader();
+
+        /**
+         * Notifies the listener about a changed leader information for the given component.
+         *
+         * @param componentId identifying the component whose leader information has changed
+         * @param leaderInformation new leader information
+         */
+        void notifyLeaderInformationChange(String componentId, LeaderInformation leaderInformation);
+
+        /**
+         * Notifies the listener about all currently known leader information.
+         *
+         * @param leaderInformationWithComponentIds leader information with component ids
+         */
+        void notifyAllKnownLeaderInformation(

Review comment:
       Yes, the different implementations manifest in different interface methods :-(




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {

Review comment:
       Yes and probably not. Will change 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            getLeaderCallback().notLeader();
+                            leaderElectionListener.await(LeaderElectionEvent.NotLeaderEvent.class);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testPublishLeaderInformation() throws Exception {}
+    public void testPublishLeaderInformation() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            final LeaderInformation leaderInformation =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            final String componentId = "foobar";
+
+                            final DefaultLeaderRetrievalService leaderRetrievalService =
+                                    new DefaultLeaderRetrievalService(
+                                            new KubernetesMultipleComponentLeaderRetrievalDriverFactory(
+                                                    getFlinkKubeClient(),
+                                                    getConfigMapSharedWatcher(),
+                                                    testExecutorExtension.getExecutor(),
+                                                    LEADER_CONFIGMAP_NAME,
+                                                    componentId));
+
+                            final TestingListener leaderRetrievalListener = new TestingListener();
+                            leaderRetrievalService.start(leaderRetrievalListener);
+
+                            leaderElectionDriver.publishLeaderInformation(
+                                    componentId, leaderInformation);
+
+                            notifyLeaderRetrievalWatchOnModifiedConfigMap();
+
+                            leaderRetrievalListener.waitForNewLeader(10_000L);
+                            assertThat(leaderRetrievalListener.getLeader())
+                                    .isEqualTo(leaderInformation);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testLeaderInformationChange() throws Exception {}
+    public void testLeaderInformationChangeNotifiesListener() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            final String componentA = "componentA";
+                            final LeaderInformation leaderInformationA =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            final String componentB = "componentB";
+                            final LeaderInformation leaderInformationB =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            leaderElectionDriver.publishLeaderInformation(
+                                    componentA, leaderInformationA);
+                            leaderElectionDriver.publishLeaderInformation(
+                                    componentB, leaderInformationB);
+
+                            notifyLeaderElectionWatchOnModifiedConfigMap();
+
+                            final LeaderElectionEvent.AllKnownLeaderInformationEvent
+                                    allKnownLeaderInformationEvent =
+                                            leaderElectionListener.await(
+                                                    LeaderElectionEvent
+                                                            .AllKnownLeaderInformationEvent.class);
+
+                            assertThat(
+                                            allKnownLeaderInformationEvent
+                                                    .getLeaderInformationWithComponentIds())
+                                    .contains(

Review comment:
       True. Will change 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -141,20 +130,8 @@ public void testRecoverFromSavepoint() throws Exception {
         jobGraphWithSavepoint.setSavepointRestoreSettings(
                 SavepointRestoreSettings.forPath(savepoint2Path));
         clusterClient.submitJob(jobGraphWithSavepoint).get(TIMEOUT, TimeUnit.MILLISECONDS);
-        CommonTestUtils.waitUntilCondition(
-                () -> clusterClient.getJobStatus(jobId).get() == JobStatus.RUNNING,
-                Deadline.fromNow(Duration.ofMillis(TIMEOUT)),
-                1000);
 
-        // The savepoint 2 should be added to jobmanager leader ConfigMap
-        final String jobManagerConfigMapName = CLUSTER_ID + "-" + jobId + "-jobmanager-leader";
-        final Optional<KubernetesConfigMap> optional =
-                flinkKubeClient.getConfigMap(jobManagerConfigMapName);
-        assertThat(optional.isPresent(), is(true));
-        final String checkpointIdKey =
-                KubernetesCheckpointStoreUtil.INSTANCE.checkpointIDToName(2L);
-        assertThat(optional.get().getData().get(checkpointIdKey), is(notNullValue()));
-        assertThat(optional.get().getData().get(Constants.CHECKPOINT_COUNTER_KEY), is("3"));
+        assertThat(clusterClient.requestJobResult(jobId).join().isSuccess()).isTrue();

Review comment:
       I'll try to do this once all the fixup commits are squashed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java
##########
@@ -53,6 +51,9 @@
  */
 public class KubernetesLeaderElectionAndRetrievalITCase extends TestLogger {

Review comment:
       I think this is a potential follow up task. Otherwise I fear that we are unnecessarily letting the scope of this PR grow.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java
##########
@@ -53,6 +51,9 @@
  */
 public class KubernetesLeaderElectionAndRetrievalITCase extends TestLogger {

Review comment:
       I think this is 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    private void handleStateChange(ConnectionState newState) {

Review comment:
       I would like to keep it because the `ZooKeeperLeaderElectionDriver` should ideally be removed with the next release.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503) 
   * a3c818cb7a4341243bbf7f584d327a1f51d355d9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   * c7aaf594b9767994a95d9cba1c2f7e60d385002a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {

Review comment:
       Yes. I can add the suffix if this makes things clearer.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
##########
@@ -82,6 +80,31 @@ public static HighAvailabilityServices createAvailableOrEmbeddedServices(
         }
     }
 
+    @Nonnull

Review comment:
       Yes. Will remove 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9bd3feac7400f1ad0f6e14c1d37308cc9ff95011 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179) 
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -141,20 +130,8 @@ public void testRecoverFromSavepoint() throws Exception {
         jobGraphWithSavepoint.setSavepointRestoreSettings(
                 SavepointRestoreSettings.forPath(savepoint2Path));
         clusterClient.submitJob(jobGraphWithSavepoint).get(TIMEOUT, TimeUnit.MILLISECONDS);
-        CommonTestUtils.waitUntilCondition(
-                () -> clusterClient.getJobStatus(jobId).get() == JobStatus.RUNNING,
-                Deadline.fromNow(Duration.ofMillis(TIMEOUT)),
-                1000);
 
-        // The savepoint 2 should be added to jobmanager leader ConfigMap
-        final String jobManagerConfigMapName = CLUSTER_ID + "-" + jobId + "-jobmanager-leader";
-        final Optional<KubernetesConfigMap> optional =
-                flinkKubeClient.getConfigMap(jobManagerConfigMapName);
-        assertThat(optional.isPresent(), is(true));
-        final String checkpointIdKey =
-                KubernetesCheckpointStoreUtil.INSTANCE.checkpointIDToName(2L);
-        assertThat(optional.get().getData().get(checkpointIdKey), is(notNullValue()));
-        assertThat(optional.get().getData().get(Constants.CHECKPOINT_COUNTER_KEY), is("3"));
+        assertThat(clusterClient.requestJobResult(jobId).join().isSuccess()).isTrue();

Review comment:
       I've factored the changes out into a separate commit: fd57ecac15c1d0e8baa1059276178c84736d3558.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 00e0b2324b8c72cf98105f985d40c006a43d6ddf Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437) 
   * a605d8da083d456fbbae4c5adaaea270263b2f38 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 00e0b2324b8c72cf98105f985d40c006a43d6ddf Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437) 
   * a605d8da083d456fbbae4c5adaaea270263b2f38 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a605d8da083d456fbbae4c5adaaea270263b2f38 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473) 
   * 972895e90d96516a2b8fa3722daf677e7759e266 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot commented on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] XComp commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+/**
+ * Leader election service that allows to register multiple {@link LeaderElectionEventHandler
+ * LeaderElectionEventHandlers} that are identified by different names. For each event handler it is
+ * possible to write the corresponding {@link LeaderInformation}.
+ */
+public interface MultipleComponentLeaderElectionService {
+
+    /**
+     * Closes this service.
+     *
+     * @throws Exception if the service failed to close
+     */
+    void close() throws Exception;
+
+    /**
+     * Creates a {@link LeaderElectionDriverFactory} for the given leader name.
+     *
+     * @param componentId identifying the component for which to create a leader election driver
+     *     factory
+     * @return Leader election driver factory
+     */
+    LeaderElectionDriverFactory createDriverFactory(String componentId);
+
+    /**
+     * Publishes the given leader information for the component identified by the given leader name.
+     *
+     * @param componentId identifying the component
+     * @param leaderInformation leader information
+     */
+    void publishLeaderInformation(String componentId, LeaderInformation leaderInformation);
+
+    /**
+     * Registers a new leader election event handler under the given component id.
+     *
+     * @param componentId identifying the leader election event handler
+     * @param leaderElectionEventHandler leader election event handler to register
+     */
+    void registerLeaderElectionEventHandler(
+            String componentId, LeaderElectionEventHandler leaderElectionEventHandler);
+
+    /**
+     * Unregisters the leader election event handler with the given component id.
+     *
+     * @param componentId identifying the component
+     * @throws Exception if the leader election event handler could not be unregistered
+     */
+    void unregisterLeaderElectionEventHandler(String componentId) throws Exception;
+
+    /**
+     * Returns whether the given component has leadership.
+     *
+     * @param componentId identifying the component
+     * @return {@code true} if the comopnent has leadership otherwise {@code false}

Review comment:
       There's a typo `comopnent`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+/**
+ * Leader election service that allows to register multiple {@link LeaderElectionEventHandler
+ * LeaderElectionEventHandlers} that are identified by different names. For each event handler it is
+ * possible to write the corresponding {@link LeaderInformation}.
+ */
+public interface MultipleComponentLeaderElectionService {

Review comment:
       Should we be more specific about the contract for each of this interface's methods when there's an invalid parameter passed (e.g. throwing `IllegalStateExceptions` if the `componentId` already exists when calling `registerLeaderElectionEventHandler`)?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionDriverFactory.java
##########
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 {@link MultipleComponentLeaderElectionDriver}. */
+public interface MultipleComponentLeaderElectionDriverFactory {
+
+    /**
+     * Creates a {@link MultipleComponentLeaderElectionDriver} for the given leader contender
+     * description. Moreover, it registers the given leader election listener with the service.

Review comment:
       ```
   Moreover, it registers the given leader election listener with the service.
   ```
   This seems to be only the case for the [MultipleComponentLeaderElectionDriverAdapter:36](https://github.com/apache/flink/blob/a0d8254b0ca7f785b70fb0686ef23d2104c771f0/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionDriverAdapter.java#L36). It's not modelled in a generic fashion into the code. Is that done on purpose? 
   
   It feels like the JavaDoc is misleading here. AFAIU, the registration is only necessary for the `MultipleComponentLeaderElectionDriverAdapter`. Hence, this functionality should be documented in that class' JavaDoc instead.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingMultipleComponentLeaderElectionDriver.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.runtime.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.util.Optional;
+
+/** Testing implementation of {@link MultipleComponentLeaderElectionDriver}. */
+public class TestingMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private final BiConsumerWithException<String, LeaderInformation, Exception>
+            publishLeaderInformationConsumer;
+    private final ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer;
+    private boolean hasLeadership;
+
+    private Optional<Listener> listener;

Review comment:
       Is it because of my personal settings or does Intellij complain in general about internally used Optionals?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull

Review comment:
       Isn't that obsolete? My understanding was that we would only annotate cases where `null` could be used.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -96,13 +106,17 @@
     /** The prefix of the completed checkpoint file. */
     public static final String HA_STORAGE_COMPLETED_CHECKPOINT = "completedCheckpoint";
 
-    private static final String RESOURCE_MANAGER_LEADER = "/resource_manager";
+    private static final String RESOURCE_MANAGER_LEADER = "resource_manager";
 
-    private static final String DISPATCHER_LEADER = "/dispatcher";
+    private static final String DISPATCHER_LEADER = "dispatcher";
 
     private static final String LEADER_NODE = "/leader";
 
-    private static final String REST_SERVER_LEADER = "/rest_server";
+    private static final String REST_SERVER_LEADER = "rest_server";
+
+    private static final String LEADER_LATCH_NODE = "/latch";

Review comment:
       Shouldn't we name this `LEADER_LATCH_NODE_SUBPATH` or remove the slash from the value analogously to the other constants and utilize `ZooKeeperUtils.generateZookeeperPath` in the corresponding getter method?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {

Review comment:
       I was wondering whether we should move it into `AbstractZooKeeperHaSerrvices`. But I agree that it is better off here because of the semantically close relation to the leader election.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;

Review comment:
       non-null check could be added

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                new ZooKeeperMultipleComponentLeaderElectionDriver(
+                        curatorFramework, "foobar", leaderElectionListener);
+        return leaderElectionDriver;
+    }
+
+    @Nonnull
+    private CuratorFrameworkWithUnhandledErrorListener startCuratorFramework() {

Review comment:
       I feel like this method should go into `ZooKeeperExtension` as it would be useful to access the ZooKeeper test instance provided by the `ZooKeeperExtension`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {

Review comment:
       We're expecting to gain leadership because of no competing contender being available, aren't we? I'm wondering whether we could make that more explicit in the test name like `testElectionDriverGainsLeadershipAtStartup` or whether that's just obvious... 🤔 
   
   For me, I was just initially puzzled why we can immediately expect to gain leadership without doing anything.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(

Review comment:
       nit: static import?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());

Review comment:
       nit: this and the try/finally block could be moved into its own method with the test logic being passed in as a callback function to reduce code redundancy

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Collection;
+
+/**
+ * A leader election driver that allows to write {@link LeaderInformation} for multiple components.
+ */
+public interface MultipleComponentLeaderElectionDriver {
+
+    /**
+     * Closes the driver.
+     *
+     * @throws Exception if closing this driver fails
+     */
+    void close() throws Exception;
+
+    /**
+     * Returns whether the driver has currently leadership.
+     *
+     * @return {@code true} if the driver has leadership, otherwise {@code false}
+     */
+    boolean hasLeadership();
+
+    /**
+     * Publishes the leader information for the given component.
+     *
+     * @param componentId identifying the component for which to publish the leader information
+     * @param leaderInformation leader information of the respective component
+     * @throws Exception if publishing fails
+     */
+    void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception;
+
+    /**
+     * Deletes the leader information for the given component.
+     *
+     * @param componentId identifying the component for which to delete the leader information
+     * @throws Exception if deleting fails
+     */
+    void deleteLeaderInformation(String componentId) throws Exception;
+
+    /**
+     * Listener interface for state changes of the {@link MultipleComponentLeaderElectionDriver}.
+     */
+    interface Listener {
+
+        /** Callback that is called once the driver obtains the leadership. */
+        void isLeader();
+
+        /** Callback that is called once the driver loses the leadership. */
+        void notLeader();
+
+        /**
+         * Notifies the listener about a changed leader information for the given component.
+         *
+         * @param componentId identifying the component whose leader information has changed
+         * @param leaderInformation new leader information
+         */
+        void notifyLeaderInformationChange(String componentId, LeaderInformation leaderInformation);
+
+        /**
+         * Notifies the listener about all currently known leader information.
+         *
+         * @param leaderInformationWithComponentIds leader information with component ids
+         */
+        void notifyAllKnownLeaderInformation(

Review comment:
       Just for clarification: Initially, I thought that the calling upstream method in the k8s implementation is only triggered on modified `ConfigMaps`. I wanted to point out that the method name is misleading. But we're using one `ConfigMap` for all the stored leader information. That's why, all the information is delived and, therefore, the name is correct.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);
+
+            ExecutorUtils.gracefulShutdown(10L, TimeUnit.SECONDS, leadershipOperationExecutor);
+
+            Exception exception = null;
+            try {
+                multipleComponentLeaderElectionDriver.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public LeaderElectionDriverFactory createDriverFactory(String leaderName) {
+        return new MultipleComponentLeaderElectionDriverAdapterFactory(leaderName, this);
+    }
+
+    @Override
+    public void publishLeaderInformation(String leaderName, LeaderInformation leaderInformation) {
+        try {
+            multipleComponentLeaderElectionDriver.publishLeaderInformation(
+                    leaderName, leaderInformation);
+        } catch (Exception e) {
+            fatalErrorHandler.onFatalError(
+                    new FlinkException(
+                            String.format(
+                                    "Could not write leader information %s for leader %s.",
+                                    leaderInformation, leaderName),
+                            e));
+        }
+    }
+
+    @Override
+    public void registerLeaderElectionEventHandler(
+            String componentId, LeaderElectionEventHandler leaderElectionEventHandler) {
+
+        synchronized (lock) {
+            Preconditions.checkState(
+                    !leaderElectionEventHandlers.containsKey(componentId),
+                    "Do not support duplicate LeaderElectionEventHandler registration under %s",
+                    componentId);
+            leaderElectionEventHandlers.put(componentId, leaderElectionEventHandler);
+
+            if (currentLeaderSessionId != null) {
+                leadershipOperationExecutor.execute(
+                        () -> leaderElectionEventHandler.onGrantLeadership(currentLeaderSessionId));
+            }
+        }
+    }
+
+    @Override
+    public void unregisterLeaderElectionEventHandler(String componentId) throws Exception {
+        final LeaderElectionEventHandler unregisteredLeaderElectionEventHandler;
+        synchronized (lock) {
+            unregisteredLeaderElectionEventHandler =
+                    leaderElectionEventHandlers.remove(componentId);
+
+            if (unregisteredLeaderElectionEventHandler != null) {

Review comment:
       Is it a too common case that a component id does not exist to have it being logged? 🤔 Otherwise, we could add a log in the else branch here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);
+
+            ExecutorUtils.gracefulShutdown(10L, TimeUnit.SECONDS, leadershipOperationExecutor);
+
+            Exception exception = null;
+            try {
+                multipleComponentLeaderElectionDriver.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public LeaderElectionDriverFactory createDriverFactory(String leaderName) {
+        return new MultipleComponentLeaderElectionDriverAdapterFactory(leaderName, this);
+    }
+
+    @Override
+    public void publishLeaderInformation(String leaderName, LeaderInformation leaderInformation) {
+        try {
+            multipleComponentLeaderElectionDriver.publishLeaderInformation(
+                    leaderName, leaderInformation);
+        } catch (Exception e) {
+            fatalErrorHandler.onFatalError(
+                    new FlinkException(
+                            String.format(
+                                    "Could not write leader information %s for leader %s.",
+                                    leaderInformation, leaderName),
+                            e));
+        }
+    }
+
+    @Override
+    public void registerLeaderElectionEventHandler(
+            String componentId, LeaderElectionEventHandler leaderElectionEventHandler) {
+
+        synchronized (lock) {
+            Preconditions.checkState(
+                    !leaderElectionEventHandlers.containsKey(componentId),
+                    "Do not support duplicate LeaderElectionEventHandler registration under %s",
+                    componentId);
+            leaderElectionEventHandlers.put(componentId, leaderElectionEventHandler);
+
+            if (currentLeaderSessionId != null) {
+                leadershipOperationExecutor.execute(
+                        () -> leaderElectionEventHandler.onGrantLeadership(currentLeaderSessionId));
+            }
+        }
+    }
+
+    @Override
+    public void unregisterLeaderElectionEventHandler(String componentId) throws Exception {
+        final LeaderElectionEventHandler unregisteredLeaderElectionEventHandler;
+        synchronized (lock) {
+            unregisteredLeaderElectionEventHandler =
+                    leaderElectionEventHandlers.remove(componentId);
+
+            if (unregisteredLeaderElectionEventHandler != null) {
+                leadershipOperationExecutor.execute(
+                        unregisteredLeaderElectionEventHandler::onRevokeLeadership);
+            }
+        }
+
+        multipleComponentLeaderElectionDriver.deleteLeaderInformation(componentId);
+    }
+
+    @Override
+    public boolean hasLeadership(String leaderName) {
+        synchronized (lock) {
+            Preconditions.checkState(running);
+
+            return leaderElectionEventHandlers.containsKey(leaderName)
+                    && multipleComponentLeaderElectionDriver.hasLeadership();
+        }
+    }
+
+    @Override
+    public void isLeader() {
+        final UUID newLeaderSessionId = UUID.randomUUID();
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            currentLeaderSessionId = UUID.randomUUID();
+
+            forEachLeaderElectionEventHandler(
+                    leaderElectionEventHandler ->
+                            leaderElectionEventHandler.onGrantLeadership(newLeaderSessionId));
+        }
+    }
+
+    @Override
+    public void notLeader() {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            currentLeaderSessionId = null;
+
+            forEachLeaderElectionEventHandler(LeaderElectionEventHandler::onRevokeLeadership);
+        }
+    }
+
+    @GuardedBy("lock")
+    private void forEachLeaderElectionEventHandler(
+            Consumer<? super LeaderElectionEventHandler> action) {
+
+        for (LeaderElectionEventHandler leaderElectionEventHandler :
+                leaderElectionEventHandlers.values()) {
+            leadershipOperationExecutor.execute(() -> action.accept(leaderElectionEventHandler));
+        }
+    }
+
+    @Override
+    public void notifyLeaderInformationChange(
+            String leaderName, LeaderInformation leaderInformation) {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            final LeaderElectionEventHandler leaderElectionEventHandler =
+                    leaderElectionEventHandlers.get(leaderName);
+
+            if (leaderElectionEventHandler != null) {
+                leadershipOperationExecutor.execute(
+                        () ->
+                                leaderElectionEventHandler.onLeaderInformationChange(
+                                        leaderInformation));
+            }
+        }
+    }
+
+    @Override
+    public void notifyAllKnownLeaderInformation(
+            Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            final Map<String, LeaderInformation> leaderInformationByName =
+                    leaderInformationWithComponentIds.stream()
+                            .collect(
+                                    Collectors.toMap(
+                                            LeaderInformationWithComponentId::getComponentId,
+                                            LeaderInformationWithComponentId
+                                                    ::getLeaderInformation));
+
+            for (Map.Entry<String, LeaderElectionEventHandler>
+                    leaderNameLeaderElectionEventHandlerPair :
+                            leaderElectionEventHandlers.entrySet()) {
+                final String leaderName = leaderNameLeaderElectionEventHandlerPair.getKey();
+                if (leaderInformationByName.containsKey(leaderName)) {
+                    leaderNameLeaderElectionEventHandlerPair
+                            .getValue()
+                            .onLeaderInformationChange(leaderInformationByName.get(leaderName));

Review comment:
       Initially, I was confused that we call the `onLeaderInformationChange` method even in cases where the leader information might not have changed for the given `componentId`. But I verified that [LeaderElectionEventHandler.onLeaderInformationChange](https://github.com/apache/flink/blob/040b903cb67fd62fdf4117b2aca34f8a2bb98c6e/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionEventHandler.java#L55) actually should cover this case internally according to the JavaDoc. 👍 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;

Review comment:
       Missing `@GuardedBy("lock")`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingMultipleComponentLeaderElectionDriver.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.runtime.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.util.Optional;
+
+/** Testing implementation of {@link MultipleComponentLeaderElectionDriver}. */
+public class TestingMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private final BiConsumerWithException<String, LeaderInformation, Exception>
+            publishLeaderInformationConsumer;
+    private final ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer;
+    private boolean hasLeadership;
+
+    private Optional<Listener> listener;
+
+    private TestingMultipleComponentLeaderElectionDriver(
+            BiConsumerWithException<String, LeaderInformation, Exception>
+                    publishLeaderInformationConsumer,
+            ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer) {
+        this.publishLeaderInformationConsumer = publishLeaderInformationConsumer;
+        this.deleteLeaderInformationConsumer = deleteLeaderInformationConsumer;
+        hasLeadership = false;
+        listener = Optional.empty();
+    }
+
+    public void grantLeadership() {
+        if (!hasLeadership) {
+            hasLeadership = true;
+            listener.ifPresent(Listener::isLeader);
+        }
+    }
+
+    public void revokeLeadership() {
+        if (hasLeadership) {
+            hasLeadership = false;
+            listener.ifPresent(Listener::notLeader);
+        }
+    }
+
+    public void setListener(Listener listener) {
+        Preconditions.checkState(!this.listener.isPresent(), "Can only set a single listener.");
+        this.listener = Optional.of(listener);
+    }
+
+    @Override
+    public void close() throws Exception {}
+
+    @Override
+    public boolean hasLeadership() {
+        return hasLeadership;
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        publishLeaderInformationConsumer.accept(componentId, leaderInformation);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String componentId) throws Exception {
+        deleteLeaderInformationConsumer.accept(componentId);
+    }
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    public static final class Builder {
+        private BiConsumerWithException<String, LeaderInformation, Exception>
+                publishLeaderInformationConsumer = (ignoredA, ignoredB) -> {};
+        private ThrowingConsumer<String, Exception> deleteLeaderInformationConsumer =
+                (ignored) -> {};

Review comment:
       nit: The brackets in `(ignored)` are not necessary

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {

Review comment:
       can't we move the `synchronized` block into the `getOrInitialize...` method? Additionally, do we gain any benefits from storing it in a local variable?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull
+    private DefaultMultipleComponentLeaderElectionService
+            createDefaultMultiplexingLeaderElectionService(
+                    TestingMultipleComponentLeaderElectionDriver leaderElectionDriver)
+                    throws Exception {
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                new DefaultMultipleComponentLeaderElectionService(
+                        fatalErrorHandlerExtension.getTestingFatalErrorHandler(),
+                        "foobar",
+                        new TestingMultipleComponentLeaderElectionDriverFactory(
+                                leaderElectionDriver),
+                        Executors.newDirectExecutorService());
+        return leaderElectionService;
+    }
+
+    @Test
+    public void notLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+            leaderElectionDriver.revokeLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isFalse();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void unregisteredEventHandlersAreNotNotified() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            final String componentId = "foobar";
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    componentId, leaderElectionEventHandler);
+            leaderElectionService.unregisterLeaderElectionEventHandler(componentId);
+
+            leaderElectionDriver.grantLeadership();
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isFalse();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void newlyRegisteredEventHandlersAreInformedAboutLeadership() throws Exception {

Review comment:
       Should we add a test checking that the onRevoke method is called when unregistering a component?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+
+import javax.annotation.Nonnull;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Abstract ZooKeeper based HA services. */
+public abstract class AbstractZooKeeperHaServices extends AbstractHaServices {
+    /** The curator resource to use. */
+    private final CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper;
+
+    public AbstractZooKeeperHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Executor executor,
+            Configuration configuration,
+            BlobStoreService blobStoreService) {
+        super(configuration, executor, blobStoreService);
+        this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper);
+    }
+
+    protected final CuratorFramework getCuratorFramework() {
+        return curatorFrameworkWrapper.asCuratorFramework();
+    }
+
+    @Override
+    public CheckpointRecoveryFactory createCheckpointRecoveryFactory() throws Exception {
+        return new ZooKeeperCheckpointRecoveryFactory(
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        curatorFrameworkWrapper.asCuratorFramework(), ZooKeeperUtils.getJobsPath()),
+                configuration,
+                ioExecutor);
+    }
+
+    @Override
+    public JobGraphStore createJobGraphStore() throws Exception {
+        return ZooKeeperUtils.createJobGraphs(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    public RunningJobsRegistry createRunningJobsRegistry() {
+        return new ZooKeeperRunningJobsRegistry(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        curatorFrameworkWrapper.close();
+    }
+
+    @Override
+    protected void internalCleanup() throws Exception {
+        cleanupZooKeeperPaths();
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        deleteZNode(ZooKeeperUtils.getLeaderPathForJob(jobID));
+    }
+
+    /** Cleans up leftover ZooKeeper paths. */
+    private void cleanupZooKeeperPaths() throws Exception {
+        deleteOwnedZNode();
+        tryDeleteEmptyParentZNodes();
+    }
+
+    private void deleteOwnedZNode() throws Exception {
+        deleteZNode("/");
+    }
+
+    protected void deleteZNode(String path) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFrameworkWrapper.asCuratorFramework(), path);
+    }
+
+    /**
+     * Tries to delete empty parent znodes.
+     *
+     * <p>IMPORTANT: This method can be removed once all supported ZooKeeper versions support the
+     * container {@link org.apache.zookeeper.CreateMode}.
+     *
+     * @throws Exception if the deletion fails for other reason than {@link
+     *     KeeperException.NotEmptyException}
+     */
+    private void tryDeleteEmptyParentZNodes() throws Exception {
+        // try to delete the parent znodes if they are empty
+        String remainingPath =
+                getParentPath(
+                        getNormalizedPath(
+                                curatorFrameworkWrapper.asCuratorFramework().getNamespace()));
+        final CuratorFramework nonNamespaceClient =
+                curatorFrameworkWrapper.asCuratorFramework().usingNamespace(null);
+
+        while (!isRootPath(remainingPath)) {
+            try {
+                nonNamespaceClient.delete().forPath(remainingPath);
+            } catch (KeeperException.NotEmptyException ignored) {
+                // We can only delete empty znodes
+                break;
+            }
+
+            remainingPath = getParentPath(remainingPath);
+        }
+    }
+
+    private static boolean isRootPath(String remainingPath) {
+        return ZKPaths.PATH_SEPARATOR.equals(remainingPath);
+    }
+
+    @Nonnull
+    private static String getNormalizedPath(String path) {
+        return ZKPaths.makePath(path, "");
+    }
+
+    @Nonnull

Review comment:
       obsolete

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull
+    private DefaultMultipleComponentLeaderElectionService
+            createDefaultMultiplexingLeaderElectionService(
+                    TestingMultipleComponentLeaderElectionDriver leaderElectionDriver)
+                    throws Exception {
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                new DefaultMultipleComponentLeaderElectionService(
+                        fatalErrorHandlerExtension.getTestingFatalErrorHandler(),
+                        "foobar",
+                        new TestingMultipleComponentLeaderElectionDriverFactory(
+                                leaderElectionDriver),
+                        Executors.newDirectExecutorService());
+        return leaderElectionService;
+    }
+
+    @Test
+    public void notLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+            leaderElectionDriver.revokeLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isFalse();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void unregisteredEventHandlersAreNotNotified() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            final String componentId = "foobar";
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    componentId, leaderElectionEventHandler);
+            leaderElectionService.unregisterLeaderElectionEventHandler(componentId);
+
+            leaderElectionDriver.grantLeadership();
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isFalse();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void newlyRegisteredEventHandlersAreInformedAboutLeadership() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            leaderElectionDriver.grantLeadership();
+
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    "foobar", leaderElectionEventHandler);
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isTrue();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void allKnownLeaderInformationCallsEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            leaderElectionDriver.grantLeadership();
+
+            final Collection<Component> knownLeaderInformation = createComponents(3);
+            final Collection<Component> unknownLeaderInformation = createComponents(2);

Review comment:
       What's the point of instantiating this here? 🤔 this collection never gets involved. We're only testing the test implementation here...

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);
+
+            ExecutorUtils.gracefulShutdown(10L, TimeUnit.SECONDS, leadershipOperationExecutor);
+
+            Exception exception = null;
+            try {
+                multipleComponentLeaderElectionDriver.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            ExceptionUtils.tryRethrowException(exception);

Review comment:
       Am I missing something here or could we just remove the try/catch logic here and just forward the exception? 🤔 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);

Review comment:
       Should we add a more descriptive log message here? There is no `toString` implementation for `DefaultMultipleComponentLeaderElectionService` for now. Or is the memory address of the instance good enough? 🤔 

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull
+    private DefaultMultipleComponentLeaderElectionService
+            createDefaultMultiplexingLeaderElectionService(
+                    TestingMultipleComponentLeaderElectionDriver leaderElectionDriver)
+                    throws Exception {
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =

Review comment:
       nit: We could return the instance here instead of creating a local variable.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionServiceTest.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.runtime.util.TestingFatalErrorHandlerExtension;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link DefaultMultipleComponentLeaderElectionService}. */
+@ExtendWith(TestLoggerExtension.class)
+class DefaultMultipleComponentLeaderElectionServiceTest {
+
+    @RegisterExtension
+    public final TestingFatalErrorHandlerExtension fatalErrorHandlerExtension =
+            new TestingFatalErrorHandlerExtension();
+
+    @Test
+    public void isLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isTrue();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Nonnull
+    private DefaultMultipleComponentLeaderElectionService
+            createDefaultMultiplexingLeaderElectionService(
+                    TestingMultipleComponentLeaderElectionDriver leaderElectionDriver)
+                    throws Exception {
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                new DefaultMultipleComponentLeaderElectionService(
+                        fatalErrorHandlerExtension.getTestingFatalErrorHandler(),
+                        "foobar",
+                        new TestingMultipleComponentLeaderElectionDriverFactory(
+                                leaderElectionDriver),
+                        Executors.newDirectExecutorService());
+        return leaderElectionService;
+    }
+
+    @Test
+    public void notLeaderInformsAllRegisteredLeaderElectionEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final Collection<SimpleTestingLeaderElectionEventListener> eventListeners =
+                    Stream.generate(SimpleTestingLeaderElectionEventListener::new)
+                            .limit(4)
+                            .collect(Collectors.toList());
+
+            int counter = 0;
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        String.valueOf(counter), eventListener);
+                counter++;
+            }
+
+            leaderElectionDriver.grantLeadership();
+            leaderElectionDriver.revokeLeadership();
+
+            for (SimpleTestingLeaderElectionEventListener eventListener : eventListeners) {
+                assertThat(eventListener.hasLeadership()).isFalse();
+            }
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void unregisteredEventHandlersAreNotNotified() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            final String componentId = "foobar";
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    componentId, leaderElectionEventHandler);
+            leaderElectionService.unregisterLeaderElectionEventHandler(componentId);
+
+            leaderElectionDriver.grantLeadership();
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isFalse();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void newlyRegisteredEventHandlersAreInformedAboutLeadership() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            leaderElectionDriver.grantLeadership();
+
+            final SimpleTestingLeaderElectionEventListener leaderElectionEventHandler =
+                    new SimpleTestingLeaderElectionEventListener();
+            leaderElectionService.registerLeaderElectionEventHandler(
+                    "foobar", leaderElectionEventHandler);
+
+            assertThat(leaderElectionEventHandler.hasLeadership()).isTrue();
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    @Test
+    public void allKnownLeaderInformationCallsEventHandlers() throws Exception {
+        final TestingMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                TestingMultipleComponentLeaderElectionDriver.newBuilder().build();
+        final DefaultMultipleComponentLeaderElectionService leaderElectionService =
+                createDefaultMultiplexingLeaderElectionService(leaderElectionDriver);
+
+        try {
+            leaderElectionDriver.grantLeadership();
+
+            final Collection<Component> knownLeaderInformation = createComponents(3);
+            final Collection<Component> unknownLeaderInformation = createComponents(2);
+
+            for (Component component : knownLeaderInformation) {
+                leaderElectionService.registerLeaderElectionEventHandler(
+                        component.getComponentId(), component.getLeaderElectionEventListener());
+            }
+
+            leaderElectionService.notifyAllKnownLeaderInformation(
+                    knownLeaderInformation.stream()
+                            .map(
+                                    component ->
+                                            LeaderInformationWithComponentId.create(
+                                                    component.getComponentId(),
+                                                    component.getLeaderInformation()))
+                            .collect(Collectors.toList()));
+
+            for (Component component : knownLeaderInformation) {
+                assertThat(component.getLeaderElectionEventListener().getLeaderInformation())
+                        .isEqualTo(component.getLeaderInformation());
+            }
+
+            for (Component component : unknownLeaderInformation) {
+                assertThat(component.getLeaderElectionEventListener().getLeaderInformation())
+                        .isNull();
+            }
+
+        } finally {
+            leaderElectionService.close();
+        }
+    }
+
+    private Collection<Component> createComponents(int numberComponents) {
+        final List<Component> result = new ArrayList<>();
+
+        for (int i = 0; i < numberComponents; i++) {
+            result.add(
+                    new Component(
+                            UUID.randomUUID().toString(),
+                            new SimpleTestingLeaderElectionEventListener(),
+                            LeaderInformation.known(UUID.randomUUID(), "localhost")));
+        }
+
+        return result;
+    }
+
+    private static final class Component {
+        private final String componentId;
+        private final SimpleTestingLeaderElectionEventListener leaderElectionEventListener;
+        private final LeaderInformation leaderInformation;
+
+        private Component(
+                String componentId,
+                SimpleTestingLeaderElectionEventListener leaderElectionEventListener,
+                LeaderInformation leaderInformation) {
+            this.componentId = componentId;
+            this.leaderElectionEventListener = leaderElectionEventListener;
+            this.leaderInformation = leaderInformation;
+        }
+
+        String getComponentId() {
+            return componentId;
+        }
+
+        LeaderInformation getLeaderInformation() {
+            return leaderInformation;
+        }
+
+        SimpleTestingLeaderElectionEventListener getLeaderElectionEventListener() {
+            return leaderElectionEventListener;
+        }
+    }
+
+    private static final class SimpleTestingLeaderElectionEventListener
+            implements LeaderElectionEventHandler {
+
+        private boolean hasLeadership;
+
+        @Nullable private UUID leaderSessionId;

Review comment:
       unused member

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {
+        // Maybe use a single service for leader retrieval
+        return ZooKeeperUtils.createLeaderRetrievalService(
+                leaderNamespacedCuratorFramework, leaderPath, configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        try {
+            super.internalClose();
+        } catch (Exception e) {
+            exception = ExceptionUtils.firstOrSuppressed(e, exception);
+        }
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        super.internalCleanupJobData(jobID);
+    }

Review comment:
       the method itself doesn't bring any added value. I guess we can remove it...

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+
+import javax.annotation.Nonnull;
+
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Abstract ZooKeeper based HA services. */
+public abstract class AbstractZooKeeperHaServices extends AbstractHaServices {
+    /** The curator resource to use. */
+    private final CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper;
+
+    public AbstractZooKeeperHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Executor executor,
+            Configuration configuration,
+            BlobStoreService blobStoreService) {
+        super(configuration, executor, blobStoreService);
+        this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper);
+    }
+
+    protected final CuratorFramework getCuratorFramework() {
+        return curatorFrameworkWrapper.asCuratorFramework();
+    }
+
+    @Override
+    public CheckpointRecoveryFactory createCheckpointRecoveryFactory() throws Exception {
+        return new ZooKeeperCheckpointRecoveryFactory(
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        curatorFrameworkWrapper.asCuratorFramework(), ZooKeeperUtils.getJobsPath()),
+                configuration,
+                ioExecutor);
+    }
+
+    @Override
+    public JobGraphStore createJobGraphStore() throws Exception {
+        return ZooKeeperUtils.createJobGraphs(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    public RunningJobsRegistry createRunningJobsRegistry() {
+        return new ZooKeeperRunningJobsRegistry(
+                curatorFrameworkWrapper.asCuratorFramework(), configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        curatorFrameworkWrapper.close();
+    }
+
+    @Override
+    protected void internalCleanup() throws Exception {
+        cleanupZooKeeperPaths();
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        deleteZNode(ZooKeeperUtils.getLeaderPathForJob(jobID));
+    }
+
+    /** Cleans up leftover ZooKeeper paths. */
+    private void cleanupZooKeeperPaths() throws Exception {
+        deleteOwnedZNode();
+        tryDeleteEmptyParentZNodes();
+    }
+
+    private void deleteOwnedZNode() throws Exception {
+        deleteZNode("/");
+    }
+
+    protected void deleteZNode(String path) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFrameworkWrapper.asCuratorFramework(), path);
+    }
+
+    /**
+     * Tries to delete empty parent znodes.
+     *
+     * <p>IMPORTANT: This method can be removed once all supported ZooKeeper versions support the
+     * container {@link org.apache.zookeeper.CreateMode}.
+     *
+     * @throws Exception if the deletion fails for other reason than {@link
+     *     KeeperException.NotEmptyException}
+     */
+    private void tryDeleteEmptyParentZNodes() throws Exception {
+        // try to delete the parent znodes if they are empty
+        String remainingPath =
+                getParentPath(
+                        getNormalizedPath(
+                                curatorFrameworkWrapper.asCuratorFramework().getNamespace()));
+        final CuratorFramework nonNamespaceClient =
+                curatorFrameworkWrapper.asCuratorFramework().usingNamespace(null);
+
+        while (!isRootPath(remainingPath)) {
+            try {
+                nonNamespaceClient.delete().forPath(remainingPath);
+            } catch (KeeperException.NotEmptyException ignored) {
+                // We can only delete empty znodes
+                break;
+            }
+
+            remainingPath = getParentPath(remainingPath);
+        }
+    }
+
+    private static boolean isRootPath(String remainingPath) {
+        return ZKPaths.PATH_SEPARATOR.equals(remainingPath);
+    }
+
+    @Nonnull

Review comment:
       obsolete

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';

Review comment:
       This member is only used in `KubernetesUtils`. Should we make it private as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(

Review comment:
       Just to understand it: The fatal error handler is used when running the cluster but not utilized when initializing it, isn't it? I was just wondering whether we should use the `fatalErrorHandler` here. But looking through the code made me conclude that it's not necessary because failure during startup are handled in the [ClusterEntrypoint#startCluster:197](https://github.com/apache/flink/blob/6b4f8a4dce08b2852cfea9db8b5dffff0712aac6/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java#L197) and handled properly in `ClusterEntrypoint.runClusterEntrypoint`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsITCase.java
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.UUID;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Integration tests for the {@link ZooKeeperUtils}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperUtilsITCase {
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testWriteAndReadLeaderInformation() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final String path = "/foobar";
+
+        try {
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "barfoo");
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformation readLeaderInformation =
+                    ZooKeeperUtils.readLeaderInformation(
+                            curatorFramework.asCuratorFramework().getData().forPath(path));
+
+            assertThat(readLeaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    @Nonnull
+    private CuratorFrameworkWithUnhandledErrorListener startCuratorFramework() {

Review comment:
       Could be moved into `ZooKeeperExtension`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -96,13 +106,17 @@
     /** The prefix of the completed checkpoint file. */
     public static final String HA_STORAGE_COMPLETED_CHECKPOINT = "completedCheckpoint";
 
-    private static final String RESOURCE_MANAGER_LEADER = "/resource_manager";
+    private static final String RESOURCE_MANAGER_LEADER = "resource_manager";
 
-    private static final String DISPATCHER_LEADER = "/dispatcher";
+    private static final String DISPATCHER_LEADER = "dispatcher";
 
     private static final String LEADER_NODE = "/leader";

Review comment:
       we also might want to remove the leading slash from `LEADER_NODE`. The slash handling happens in [ZooKeeperUtils.generateZookeeperPath](https://github.com/apache/flink/blob/23d942cb6bba947ca3844687a65e8d0451c62041/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java#L661). But that would mean that we have to change `ZooKeeperUtils.getLeaderPath` to also utilize `generateZooKeeper` which could be implemented using varargs instead...

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -96,13 +106,17 @@
     /** The prefix of the completed checkpoint file. */
     public static final String HA_STORAGE_COMPLETED_CHECKPOINT = "completedCheckpoint";
 
-    private static final String RESOURCE_MANAGER_LEADER = "/resource_manager";
+    private static final String RESOURCE_MANAGER_LEADER = "resource_manager";

Review comment:
       As already mentioned in another comment: We should come up with a clear distinction between path and node name in `ZooKeeperUtils`. Right now, we're not labeling member variables/constants/methods consistently, I feel.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionEvent.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.Collection;
+
+/** Leader election event. */
+public abstract class LeaderElectionEvent {
+    public boolean isIsLeaderEvent() {
+        return false;
+    }
+
+    public boolean isNotLeaderEvent() {
+        return false;
+    }
+
+    public boolean isLeaderInformationChangeEvent() {
+        return false;
+    }
+
+    public boolean isAllKnownLeaderInformationEvent() {
+        return false;
+    }
+
+    public IsLeaderEvent asIsLeaderEvent() {
+        return as(IsLeaderEvent.class);
+    }
+
+    public NotLeaderEvent asNotLeaderEvent() {
+        return as(NotLeaderEvent.class);
+    }
+
+    public LeaderInformationChangeEvent asLeaderInformationChangeEvent() {
+        return as(LeaderInformationChangeEvent.class);
+    }
+
+    public AllKnownLeaderInformationEvent asAllKnownLeaderInformationEvent() {
+        return as(AllKnownLeaderInformationEvent.class);
+    }

Review comment:
       I couldn't find any usage for the casting methods. What purpose do they solve here? If they do, we might want to add some documentation here. If they don't for now, we could also remove them.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';
+
+    public static String encodeLeaderInformation(LeaderInformation leaderInformation) {
+        Preconditions.checkState(!leaderInformation.isEmpty());

Review comment:
       I know that this is implicitly done by `isEmpty()`. But can't we also check for `leaderInformation.getLeaderSessionID` not being `null`? That would remove the IDE warning about missing null check. That an empty `LeaderInformation` is the only time where the session ID can be `null` is an implementation detail of `LeaderInformation` which shouldn't be a concern of this method, I think.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {
+        // Maybe use a single service for leader retrieval
+        return ZooKeeperUtils.createLeaderRetrievalService(
+                leaderNamespacedCuratorFramework, leaderPath, configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        try {
+            super.internalClose();
+        } catch (Exception e) {
+            exception = ExceptionUtils.firstOrSuppressed(e, exception);
+        }
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        super.internalCleanupJobData(jobID);
+    }
+
+    @Override
+    protected String getLeaderPathForResourceManager() {
+        return ZooKeeperUtils.getResourceManagerNode();
+    }
+
+    @Override
+    protected String getLeaderPathForDispatcher() {
+        return ZooKeeperUtils.getDispatcherNode();
+    }
+
+    @Override
+    protected String getLeaderPathForJobManager(JobID jobID) {
+        return jobID.toString();
+    }
+
+    @Override
+    protected String getLeaderPathForRestServer() {
+        return ZooKeeperUtils.getRestServerNode();
+    }

Review comment:
       Is it intentional that we return values without a leading slash here as "paths"? I feel like we make a clear distinction between the node name and the path (or subpath) in `ZooKeeperUtils`) to make the intention clearer. To me, it looks like we should provide a leading slash here since the methods of the downstream call hierarchy also talk about paths.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -555,6 +668,10 @@ public static String generateZookeeperPath(String root, String child) {
         return result;
     }
 
+    public static String makeZooKeeperPath(String path) {

Review comment:
       That's essentially what `ZooKeeperUtils.generateZookeeperPath` does. We could align both methods by accepting a `varargs` list as parameters for `generateZookeeperPath`, instead. WDYT?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));

Review comment:
       Can be combined into a single call `.toArray(CompletableFuture[]::new))`

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';

Review comment:
       Shouldn't we move these two declarations up into the top of the class definition? I would expect it to be there...

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';
+
+    public static String encodeLeaderInformation(LeaderInformation leaderInformation) {
+        Preconditions.checkState(!leaderInformation.isEmpty());
+        return leaderInformation.getLeaderSessionID().toString()
+                + LEADER_INFORMATION_SEPARATOR
+                + leaderInformation.getLeaderAddress();
+    }
+
+    public static LeaderInformation parseLeaderInformationSafely(String value) {
+        try {
+            return parseLeaderInformation(value);
+        } catch (Throwable throwable) {
+            LOG.debug("Could not parse value {} into LeaderInformation.", value, throwable);
+            return LeaderInformation.empty();
+        }
+    }
+
+    private static LeaderInformation parseLeaderInformation(String value) {
+        final int splitIndex = value.indexOf(LEADER_INFORMATION_SEPARATOR);
+
+        Preconditions.checkState(splitIndex >= 0, "Expecting '<session_id>,<leader_address>'");
+
+        final UUID leaderSessionId = UUID.fromString(value.substring(0, splitIndex));
+        final String leaderAddress = value.substring(splitIndex + 1);
+
+        return LeaderInformation.known(leaderSessionId, leaderAddress);
+    }
+
+    @Nonnull
+    public static String createSingleLeaderKey(String componentId) {
+        return LEADER_PREFIX + componentId;
+    }
+
+    public static boolean isSingleLeaderKey(String key) {
+        return key.startsWith(LEADER_PREFIX);
+    }
+
+    @Nonnull

Review comment:
       nit: I'm still wondering whether the `@Nonnull` annotations are necessary here. 🤔 

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';
+
+    public static String encodeLeaderInformation(LeaderInformation leaderInformation) {
+        Preconditions.checkState(!leaderInformation.isEmpty());
+        return leaderInformation.getLeaderSessionID().toString()
+                + LEADER_INFORMATION_SEPARATOR
+                + leaderInformation.getLeaderAddress();
+    }
+
+    public static LeaderInformation parseLeaderInformationSafely(String value) {
+        try {
+            return parseLeaderInformation(value);
+        } catch (Throwable throwable) {
+            LOG.debug("Could not parse value {} into LeaderInformation.", value, throwable);
+            return LeaderInformation.empty();
+        }
+    }
+
+    private static LeaderInformation parseLeaderInformation(String value) {
+        final int splitIndex = value.indexOf(LEADER_INFORMATION_SEPARATOR);
+
+        Preconditions.checkState(splitIndex >= 0, "Expecting '<session_id>,<leader_address>'");

Review comment:
       We might want to use `LEADER_INFORMATION_SEPARATOR` in the error message instead of an hard-coded comma.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -137,17 +155,37 @@ public static String getCheckpointIdCounterPath() {
         return "/checkpoint_id_counter";
     }
 
+    public static String getLeaderPath() {
+        return LEADER_NODE;
+    }
+
+    public static String getDispatcherNode() {
+        return DISPATCHER_LEADER;
+    }
+
+    public static String getResourceManagerNode() {
+        return RESOURCE_MANAGER_LEADER;
+    }
+
+    public static String getRestServerNode() {
+        return REST_SERVER_LEADER;
+    }
+
+    public static String getLeaderLatchNode() {
+        return LEADER_LATCH_NODE;
+    }
+
     private static String getLeaderPath(String suffix) {
         return generateZookeeperPath(LEADER_NODE, suffix);
     }
 
     @Nonnull

Review comment:
       `@Nonnull` is obsolete

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
##########
@@ -500,5 +502,49 @@ public static String tryToGetPrettyPrintYaml(KubernetesResource kubernetesResour
         TASK_MANAGER
     }
 
+    private static final String LEADER_PREFIX = "org.apache.flink.k8s.leader.";
+    public static final char LEADER_INFORMATION_SEPARATOR = ',';
+
+    public static String encodeLeaderInformation(LeaderInformation leaderInformation) {
+        Preconditions.checkState(!leaderInformation.isEmpty());
+        return leaderInformation.getLeaderSessionID().toString()
+                + LEADER_INFORMATION_SEPARATOR
+                + leaderInformation.getLeaderAddress();
+    }
+
+    public static LeaderInformation parseLeaderInformationSafely(String value) {

Review comment:
       I don't like how the error case is hidden in this method. The error case should be handled explicitly in the calling method. What about returning an `Optional` here. This enables the calling method to do `parseLeaderInformationSafely("blablubb").orElse(LeaderInformation.empty())`. It's creating redundant code but improves the readability in my opinion.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",

Review comment:
       this description is used only for the purpose of logging if the connection is lost (see [ZooKeeperMultipleComponentLeaderElectionDriver:175](https://github.com/apache/flink/blob/23d942cb6bba947ca3844687a65e8d0451c62041/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java#L175). I couldn't find any code location where the `ZooKeeperMultipleComponentLeaderElectionDriver.toString` method is utilized. Either, we should add some random string here to differentiate different Dispatcher instances or we could remove it entirely (the k8s implementation doesn't make use of it at all but swallows it in `KubernetesMultipleComponentLeaderElectionDriverFactory.create`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {

Review comment:
       I need some clarification here: `event.getData()` returning `null` means that the leadership was lost, doesn't it? If yes, shouldn't that be forwarded to the `leaderElectionListener` as well?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher;
+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.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.LeaderInformationWithComponentId;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriver;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+
+/** {@link MultipleComponentLeaderElectionDriver} for Kubernetes. */
+public class KubernetesMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(KubernetesMultipleComponentLeaderElectionDriver.class);
+
+    private final FlinkKubeClient kubeClient;
+
+    private final String configMapName;
+
+    private final String lockIdentity;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final KubernetesLeaderElector leaderElector;
+
+    // Labels will be used to clean up the ha related ConfigMaps.
+    private final Map<String, String> configMapLabels;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    private final KubernetesSharedWatcher.Watch kubernetesWatch;
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public KubernetesMultipleComponentLeaderElectionDriver(
+            KubernetesLeaderElectionConfiguration leaderElectionConfiguration,
+            FlinkKubeClient kubeClient,
+            Listener leaderElectionListener,
+            KubernetesConfigMapSharedWatcher configMapSharedWatcher,
+            Executor watchExecutor,
+            FatalErrorHandler fatalErrorHandler) {
+        this.kubeClient = kubeClient;
+        this.leaderElectionListener = leaderElectionListener;
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.configMapName = leaderElectionConfiguration.getConfigMapName();
+        this.lockIdentity = leaderElectionConfiguration.getLockIdentity();
+
+        this.leaderElector =
+                kubeClient.createLeaderElector(
+                        leaderElectionConfiguration, new LeaderCallbackHandlerImpl());
+
+        this.configMapLabels =
+                KubernetesUtils.getConfigMapLabels(
+                        leaderElectionConfiguration.getClusterId(),
+                        LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+        kubernetesWatch =
+                configMapSharedWatcher.watch(
+                        configMapName, new ConfigMapCallbackHandlerImpl(), watchExecutor);
+
+        leaderElector.run();

Review comment:
       Here as well. I'd prefer extending the interface introducing a separate `start` method to have a clearer separation between instantiation and execution.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher;
+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.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.LeaderInformationWithComponentId;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriver;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+
+/** {@link MultipleComponentLeaderElectionDriver} for Kubernetes. */
+public class KubernetesMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(KubernetesMultipleComponentLeaderElectionDriver.class);
+
+    private final FlinkKubeClient kubeClient;
+
+    private final String configMapName;
+
+    private final String lockIdentity;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final KubernetesLeaderElector leaderElector;
+
+    // Labels will be used to clean up the ha related ConfigMaps.
+    private final Map<String, String> configMapLabels;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    private final KubernetesSharedWatcher.Watch kubernetesWatch;
+
+    private AtomicBoolean running = new AtomicBoolean(true);

Review comment:
       can be marked as `final`

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.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.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.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesConfigMapSharedWatcher;
+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.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+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.NAME_SEPARATOR;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Kubernetes HA services that use a single leader election service per JobManager. */
+public class KubernetesMultipleComponentLeaderElectionHaServices extends AbstractHaServices {
+
+    private final Object lock = new Object();
+
+    private final String clusterId;
+
+    private final FlinkKubeClient kubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+    private final ExecutorService watchExecutorService;
+
+    private final String lockIdentity;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private DefaultMultipleComponentLeaderElectionService multipleComponentLeaderElectionService =
+            null;
+
+    KubernetesMultipleComponentLeaderElectionHaServices(
+            FlinkKubeClient kubeClient,
+            Executor executor,
+            Configuration config,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler) {
+
+        super(config, executor, blobStoreService);
+        this.kubeClient = checkNotNull(kubeClient);
+        this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID));
+        this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+        this.configMapSharedWatcher =
+                this.kubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+        this.watchExecutorService =
+                Executors.newCachedThreadPool(
+                        new ExecutorThreadFactory("config-map-watch-handler"));
+
+        lockIdentity = UUID.randomUUID().toString();
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService =
+                getOrInitializeSingleLeaderElectionService();
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    private DefaultMultipleComponentLeaderElectionService
+            getOrInitializeSingleLeaderElectionService() {
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService == null) {
+                try {
+
+                    final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
+                            new KubernetesLeaderElectionConfiguration(
+                                    getClusterConfigMap(), lockIdentity, configuration);
+                    multipleComponentLeaderElectionService =
+                            new DefaultMultipleComponentLeaderElectionService(
+                                    fatalErrorHandler,
+                                    "Single leader election service",
+                                    new KubernetesMultipleComponentLeaderElectionDriverFactory(
+                                            kubeClient,
+                                            leaderElectionConfiguration,
+                                            configMapSharedWatcher,
+                                            watchExecutorService,
+                                            fatalErrorHandler));
+                } catch (Exception e) {
+                    throw new FlinkRuntimeException(
+                            "Could not initialize the default single leader election service.", e);
+                }
+            }
+
+            return multipleComponentLeaderElectionService;
+        }
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String componentId) {
+        return new DefaultLeaderRetrievalService(
+                new KubernetesMultipleComponentLeaderRetrievalDriverFactory(
+                        kubeClient,
+                        configMapSharedWatcher,
+                        watchExecutorService,
+                        getClusterConfigMap(),
+                        componentId));
+    }
+
+    @Override
+    protected CheckpointRecoveryFactory createCheckpointRecoveryFactory() {
+        return new KubernetesCheckpointRecoveryFactory(
+                kubeClient, configuration, ioExecutor, this::getJobSpecificConfigMap, lockIdentity);
+    }
+
+    private String getJobSpecificConfigMap(JobID jobID) {
+        return clusterId + NAME_SEPARATOR + jobID.toString() + NAME_SEPARATOR + "config-map";
+    }
+
+    @Override
+    protected JobGraphStore createJobGraphStore() throws Exception {
+        return KubernetesUtils.createJobGraphStore(
+                configuration, kubeClient, getClusterConfigMap(), lockIdentity);
+    }
+
+    private String getClusterConfigMap() {
+        return clusterId + NAME_SEPARATOR + "cluster-config-map";
+    }
+
+    @Override
+    protected RunningJobsRegistry createRunningJobsRegistry() {
+        return new KubernetesRunningJobsRegistry(kubeClient, getClusterConfigMap(), lockIdentity);
+    }
+
+    @Override
+    public void internalClose() throws Exception {
+        Exception exception = null;
+        try {
+            closeK8sServices();
+        } catch (Exception e) {
+            exception = e;
+        }
+
+        kubeClient.close();
+        ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, this.watchExecutorService);
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    private void closeK8sServices() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        configMapSharedWatcher.close();
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    public void internalCleanup() throws Exception {
+        Exception exception = null;
+        // in order to clean up, we first need to stop the services that rely on the config maps
+        try {
+            closeK8sServices();
+        } catch (Exception e) {
+            exception = e;
+        }
+
+        kubeClient
+                .deleteConfigMapsByLabels(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY))
+                .get();
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    public void internalCleanupJobData(JobID jobID) throws Exception {
+        kubeClient.deleteConfigMap(getJobSpecificConfigMap(jobID)).get();
+        // need to delete job specific leader address from leader config map
+    }
+
+    @Override
+    protected String getLeaderPathForResourceManager() {
+        return "resourcemanager";

Review comment:
       Shouldn't we move this stuff into `KubernetesUtils` as well similar to `ZooKeeperUtils`? Right now, this is redundant to `KubernetesHaServices`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (running.get() && leaderLatch.hasLeadership() && isConnectionInfoNode(childData)) {
+
+            final String path = childData.getPath();
+            final String[] splits = path.split("/");
+
+            Preconditions.checkState(
+                    splits.length >= 2,
+                    String.format(
+                            "Expecting path consisting of <leader_name>/connection_info. Got path '%s'",
+                            path));
+            final String leaderName = splits[splits.length - 2];
+
+            final LeaderInformation leaderInformation =
+                    tryReadingLeaderInformation(childData, leaderName);
+
+            leaderElectionListener.notifyLeaderInformationChange(leaderName, leaderInformation);
+        }
+    }
+
+    private boolean isConnectionInfoNode(ChildData childData) {
+        return childData.getPath().endsWith(ZooKeeperUtils.CONNECTION_INFO_NODE);
+    }
+
+    private LeaderInformation tryReadingLeaderInformation(ChildData childData, String id) {
+        LeaderInformation leaderInformation;
+        try {
+            leaderInformation = ZooKeeperUtils.readLeaderInformation(childData.getData());
+
+            LOG.debug("Leader information for {} has changed to {}.", id, leaderInformation);
+        } catch (IOException | ClassNotFoundException e) {
+            LOG.debug(
+                    "Could not read leader information for {}. Rewriting the information.", id, e);
+            leaderInformation = LeaderInformation.empty();
+        }
+
+        return leaderInformation;
+    }
+
+    private static class ConnectionInfoNodeSelector implements TreeCacheSelector {

Review comment:
       You might want to add some JavaDoc documentation here describing the purpose of that selector with a reference to the `ZooKeeperMultipleComponentLeaderElectionHaServices` JavaDoc where the node tree structure is described in detail.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(

Review comment:
       I found it misleading here that we call the method `start...` but just instantiate the driver. But I realized that we actually do start the latch and cache in the constructor. I commented on it separately.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher;
+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.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.LeaderInformationWithComponentId;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriver;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+
+/** {@link MultipleComponentLeaderElectionDriver} for Kubernetes. */
+public class KubernetesMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(KubernetesMultipleComponentLeaderElectionDriver.class);
+
+    private final FlinkKubeClient kubeClient;
+
+    private final String configMapName;
+
+    private final String lockIdentity;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final KubernetesLeaderElector leaderElector;
+
+    // Labels will be used to clean up the ha related ConfigMaps.
+    private final Map<String, String> configMapLabels;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    private final KubernetesSharedWatcher.Watch kubernetesWatch;
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public KubernetesMultipleComponentLeaderElectionDriver(
+            KubernetesLeaderElectionConfiguration leaderElectionConfiguration,
+            FlinkKubeClient kubeClient,
+            Listener leaderElectionListener,
+            KubernetesConfigMapSharedWatcher configMapSharedWatcher,
+            Executor watchExecutor,
+            FatalErrorHandler fatalErrorHandler) {
+        this.kubeClient = kubeClient;
+        this.leaderElectionListener = leaderElectionListener;
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.configMapName = leaderElectionConfiguration.getConfigMapName();
+        this.lockIdentity = leaderElectionConfiguration.getLockIdentity();
+
+        this.leaderElector =
+                kubeClient.createLeaderElector(
+                        leaderElectionConfiguration, new LeaderCallbackHandlerImpl());
+
+        this.configMapLabels =
+                KubernetesUtils.getConfigMapLabels(
+                        leaderElectionConfiguration.getClusterId(),
+                        LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+        kubernetesWatch =
+                configMapSharedWatcher.watch(
+                        configMapName, new ConfigMapCallbackHandlerImpl(), watchExecutor);
+
+        leaderElector.run();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            leaderElector.stop();
+            kubernetesWatch.close();
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        Preconditions.checkState(running.get());
+        final Optional<KubernetesConfigMap> optionalConfigMap =
+                kubeClient.getConfigMap(configMapName);
+
+        if (optionalConfigMap.isPresent()) {
+            return KubernetesLeaderElector.hasLeadership(optionalConfigMap.get(), lockIdentity);
+        } else {
+            fatalErrorHandler.onFatalError(
+                    new KubernetesException(
+                            String.format(
+                                    "ConfigMap %s does not exist. This indicates that somebody has interfered with Flink's operation.",
+                                    configMapName)));
+            return false;
+        }
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        kubeClient
+                .checkAndUpdateConfigMap(
+                        configMapName,
+                        updateConfigMapWithLeaderInformation(componentId, leaderInformation))
+                .get();
+
+        LOG.debug(
+                "Successfully wrote leader information {} for leader {} into the config map {}.",
+                leaderInformation,
+                componentId,
+                configMapName);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String componentId) throws Exception {
+        publishLeaderInformation(componentId, LeaderInformation.empty());
+    }
+
+    private Function<KubernetesConfigMap, Optional<KubernetesConfigMap>>
+            updateConfigMapWithLeaderInformation(
+                    String leaderName, LeaderInformation leaderInformation) {
+        final String configMapDataKey = KubernetesUtils.createSingleLeaderKey(leaderName);
+
+        return kubernetesConfigMap -> {
+            if (KubernetesLeaderElector.hasLeadership(kubernetesConfigMap, lockIdentity)) {
+                final Map<String, String> data = kubernetesConfigMap.getData();
+
+                if (leaderInformation.isEmpty()) {
+                    data.remove(configMapDataKey);
+                } else {
+                    data.put(
+                            configMapDataKey,
+                            KubernetesUtils.encodeLeaderInformation(leaderInformation));
+                }
+
+                kubernetesConfigMap.getLabels().putAll(configMapLabels);
+                return Optional.of(kubernetesConfigMap);
+            }
+
+            return Optional.empty();
+        };
+    }
+
+    private static Collection<LeaderInformationWithComponentId> extractLeaderInformation(
+            KubernetesConfigMap configMap) {
+        final Map<String, String> data = configMap.getData();
+
+        final Collection<LeaderInformationWithComponentId> leaderInformationWithLeaderNames =
+                new ArrayList<>();
+
+        for (Map.Entry<String, String> keyValuePair : data.entrySet()) {
+            final String key = keyValuePair.getKey();
+            if (KubernetesUtils.isSingleLeaderKey(key)) {
+                final String leaderName = KubernetesUtils.extractLeaderName(key);
+                final LeaderInformation leaderInformation =
+                        KubernetesUtils.parseLeaderInformationSafely(keyValuePair.getValue());

Review comment:
       Wouldn't it leave us in an inconsistent state if the leader information was persisted in an invalid state which leads to an empty leader information record here?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -248,5 +229,24 @@ public void run(SourceContext<Integer> ctx) throws Exception {
         public void cancel() {
             running = false;
         }
+
+        @Override
+        public void snapshotState(FunctionSnapshotContext context) throws Exception {}
+
+        @Override
+        public void initializeState(FunctionInitializationContext context) throws Exception {
+            final ListState<Integer> hasExecutedBeforeState =
+                    context.getOperatorStateStore()
+                            .getUnionListState(hasExecutedBeforeStateDescriptor);
+
+            // if we have state, then we resume from a savepoint --> stop the execution then
+            if (hasExecutedBeforeState.get().iterator().hasNext()) {
+                running = false;
+            }
+
+            hasExecutedBeforeState.clear();
+            // mark this subtask as executed before
+            hasExecutedBeforeState.add(getRuntimeContext().getIndexOfThisSubtask());

Review comment:
       The purpose of this initialization routine is to detect whether we resumed from a savepoint or not. In the former case, we would just cancel the run of the operator and make the MiniCluster stop due to the job stopping. Correct? I'm a bit confused by the `hasExecutedBeforeState` name. I don't really have a better name here, though. What about `stateFromSavepoint`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =

Review comment:
       local variable not necessary

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -248,5 +229,24 @@ public void run(SourceContext<Integer> ctx) throws Exception {
         public void cancel() {
             running = false;
         }
+
+        @Override
+        public void snapshotState(FunctionSnapshotContext context) throws Exception {}
+
+        @Override
+        public void initializeState(FunctionInitializationContext context) throws Exception {
+            final ListState<Integer> hasExecutedBeforeState =
+                    context.getOperatorStateStore()
+                            .getUnionListState(hasExecutedBeforeStateDescriptor);
+
+            // if we have state, then we resume from a savepoint --> stop the execution then
+            if (hasExecutedBeforeState.get().iterator().hasNext()) {
+                running = false;
+            }
+
+            hasExecutedBeforeState.clear();

Review comment:
       What's the purpose of clearing the state here? To limit memory? Or does it have another functional purpose I don't see. Maybe, we could add a comment if it's really only about memory consumption.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    private void handleStateChange(ConnectionState newState) {

Review comment:
       Duplicate code: Could be moved into `ZooKeeperUtils`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull

Review comment:
       obsolete `@Nonnull` annotation

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (running.get() && leaderLatch.hasLeadership() && isConnectionInfoNode(childData)) {
+
+            final String path = childData.getPath();
+            final String[] splits = path.split("/");

Review comment:
       nit: I feel like splitting up ZK paths is also something that should go into ZooKeeperUtils analogously to generating the path.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                new ZooKeeperMultipleComponentLeaderElectionDriver(
+                        curatorFramework, "foobar", leaderElectionListener);
+        return leaderElectionDriver;
+    }
+
+    @Nonnull
+    private CuratorFrameworkWithUnhandledErrorListener startCuratorFramework() {
+        final Configuration configuration = new Configuration();
+        configuration.set(
+                HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperExtension.getConnectString());
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework =

Review comment:
       local variable not necessary

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();

Review comment:
       I know that it's also how it was done in the old code and works. But shouldn't we separate the initialization and the actual starting of the instances? What about creating a static method `createAndStart` that covers both.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {

Review comment:
       shouldn't we delete the available `LeaderInformation` for the `componentId` from the ZooKeeper instance if the `leaderInformation` is empty?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                new ZooKeeperMultipleComponentLeaderElectionDriver(
+                        curatorFramework, "foobar", leaderElectionListener);
+        return leaderElectionDriver;
+    }
+
+    @Nonnull

Review comment:
       obsolete `@Nonnull`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {

Review comment:
       What about a test that checks that nothing is published if the `leaderElectionDriver` isn't elected as the leader? Additionally, what about revoking the leadership (passing in empty `LeaderInformation`)?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();

Review comment:
       nit: Renaming `TestingListener` into `TestingLeaderRetrievalListener` would improve readability of the code

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";

Review comment:
       nit: I'd prefer a more descriptive ID here. `barfoo` implies some relation to the address `foobar` which we use later on in the test. `retrieving-component` would be better here.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";

Review comment:
       same here with similar Strings being used for different purposes (address vs componentId) which might be misleading. I'd prefer descriptive Strings, instead

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -141,20 +130,8 @@ public void testRecoverFromSavepoint() throws Exception {
         jobGraphWithSavepoint.setSavepointRestoreSettings(
                 SavepointRestoreSettings.forPath(savepoint2Path));
         clusterClient.submitJob(jobGraphWithSavepoint).get(TIMEOUT, TimeUnit.MILLISECONDS);
-        CommonTestUtils.waitUntilCondition(
-                () -> clusterClient.getJobStatus(jobId).get() == JobStatus.RUNNING,
-                Deadline.fromNow(Duration.ofMillis(TIMEOUT)),
-                1000);
 
-        // The savepoint 2 should be added to jobmanager leader ConfigMap
-        final String jobManagerConfigMapName = CLUSTER_ID + "-" + jobId + "-jobmanager-leader";
-        final Optional<KubernetesConfigMap> optional =
-                flinkKubeClient.getConfigMap(jobManagerConfigMapName);
-        assertThat(optional.isPresent(), is(true));
-        final String checkpointIdKey =
-                KubernetesCheckpointStoreUtil.INSTANCE.checkpointIDToName(2L);
-        assertThat(optional.get().getData().get(checkpointIdKey), is(notNullValue()));
-        assertThat(optional.get().getData().get(Constants.CHECKPOINT_COUNTER_KEY), is("3"));
+        assertThat(clusterClient.requestJobResult(jobId).join().isSuccess()).isTrue();

Review comment:
       This looks like a refactoring of the test class. Could we move this into a separate commit? This is the test that's causing issues in [FLINK-25432](https://issues.apache.org/jira/browse/FLINK-25432). Having the refactoring done in a separate commit would help testing the test before and after the refactoring.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesTestFixture.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+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.kubernetes.kubeclient.resources.KubernetesLeaderElector;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+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.CompletionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+
+/** Test fixture for Kubernetes tests that sets up a mock {@link FlinkKubeClient}. */
+class KubernetesTestFixture {
+    private static final long TIMEOUT = 30L * 1000L;
+
+    private final String leaderConfigmapName;
+    private final String lockIdentity;
+
+    private final Configuration configuration;
+
+    private final Map<String, KubernetesConfigMap> configMapStore = new HashMap<>();
+
+    private final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+            configMapCallbackFutures = new ArrayList<>();
+
+    private final List<TestingFlinkKubeClient.MockKubernetesWatch> configMapWatches =
+            new ArrayList<>();
+
+    private final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture =
+            new CompletableFuture<>();
+    private final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+    private final CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
+            leaderCallbackHandlerFuture = new CompletableFuture<>();
+
+    private final FlinkKubeClient flinkKubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+
+    KubernetesTestFixture(String clusterId, String leaderConfigmapName, String lockIdentity) {
+        this.leaderConfigmapName = leaderConfigmapName;
+        this.lockIdentity = lockIdentity;
+        configuration = new Configuration();
+        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
+
+        flinkKubeClient = createFlinkKubeClient();
+        configMapSharedWatcher =
+                flinkKubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+    }
+
+    void close() {
+        configMapSharedWatcher.close();
+    }
+
+    FlinkKubeClient getFlinkKubeClient() {
+        return flinkKubeClient;
+    }
+
+    CompletableFuture<Void> getCloseKubeClientFuture() {
+        return closeKubeClientFuture;
+    }
+
+    CompletableFuture<Map<String, String>> getDeleteConfigMapByLabelsFuture() {
+        return deleteConfigMapByLabelsFuture;
+    }
+
+    KubernetesConfigMapSharedWatcher getConfigMapSharedWatcher() {
+        return configMapSharedWatcher;
+    }
+
+    Configuration getConfiguration() {
+        return configuration;
+    }
+
+    KubernetesConfigMap getLeaderConfigMap() {
+        final Optional<KubernetesConfigMap> configMapOpt =
+                flinkKubeClient.getConfigMap(leaderConfigmapName);
+        assertThat(configMapOpt.isPresent(), is(true));
+        return configMapOpt.get();
+    }
+
+    // Use the leader callback to manually grant leadership
+    void leaderCallbackGrantLeadership() throws Exception {
+        createLeaderConfigMap();
+        getLeaderCallback().isLeader();
+    }
+
+    FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> getLeaderElectionConfigMapCallback()
+            throws Exception {
+        assertThat(configMapCallbackFutures.size(), is(greaterThanOrEqualTo(1)));
+        return configMapCallbackFutures.get(0).get(TIMEOUT, TimeUnit.MILLISECONDS);
+    }
+
+    FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> getLeaderRetrievalConfigMapCallback()
+            throws Exception {
+        assertThat(configMapCallbackFutures.size(), is(2));
+        return configMapCallbackFutures.get(1).get(TIMEOUT, TimeUnit.MILLISECONDS);
+    }
+
+    KubernetesLeaderElector.LeaderCallbackHandler getLeaderCallback() throws Exception {
+        return leaderCallbackHandlerFuture.get(TIMEOUT, TimeUnit.MILLISECONDS);
+    }
+
+    private FlinkKubeClient createFlinkKubeClient() {
+        return createFlinkKubeClientBuilder().build();
+    }
+
+    TestingFlinkKubeClient.Builder createFlinkKubeClientBuilder() {

Review comment:
       I know that it's just copied over from `KubernetesHighAvailabilityTestBase`. But I'm wondering whether we could move the individual callables into dedicated methods? 🤔 ...not sure whether it improves readability, though.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();

Review comment:
       nit: this already tested in the `testElectionDriverGainsLeadership`

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            getLeaderCallback().notLeader();
+                            leaderElectionListener.await(LeaderElectionEvent.NotLeaderEvent.class);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testPublishLeaderInformation() throws Exception {}
+    public void testPublishLeaderInformation() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            final LeaderInformation leaderInformation =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            final String componentId = "foobar";

Review comment:
       I'd rather have different names for `componentId` and `LEADER_CONFIGMAP_NAME` to avoid confusion. Here, I'm promoting again descriptive strings 8)

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java
##########
@@ -53,6 +51,9 @@
  */
 public class KubernetesLeaderElectionAndRetrievalITCase extends TestLogger {

Review comment:
       Should we also refactor this test to use the new implementation?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            getLeaderCallback().notLeader();
+                            leaderElectionListener.await(LeaderElectionEvent.NotLeaderEvent.class);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testPublishLeaderInformation() throws Exception {}
+    public void testPublishLeaderInformation() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();

Review comment:
       nit: this is already tested in `testElectionDriverGainsLeadership`

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            getLeaderCallback().notLeader();
+                            leaderElectionListener.await(LeaderElectionEvent.NotLeaderEvent.class);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testPublishLeaderInformation() throws Exception {}
+    public void testPublishLeaderInformation() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            final LeaderInformation leaderInformation =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            final String componentId = "foobar";
+
+                            final DefaultLeaderRetrievalService leaderRetrievalService =
+                                    new DefaultLeaderRetrievalService(
+                                            new KubernetesMultipleComponentLeaderRetrievalDriverFactory(
+                                                    getFlinkKubeClient(),
+                                                    getConfigMapSharedWatcher(),
+                                                    testExecutorExtension.getExecutor(),
+                                                    LEADER_CONFIGMAP_NAME,
+                                                    componentId));
+
+                            final TestingListener leaderRetrievalListener = new TestingListener();
+                            leaderRetrievalService.start(leaderRetrievalListener);
+
+                            leaderElectionDriver.publishLeaderInformation(
+                                    componentId, leaderInformation);
+
+                            notifyLeaderRetrievalWatchOnModifiedConfigMap();
+
+                            leaderRetrievalListener.waitForNewLeader(10_000L);
+                            assertThat(leaderRetrievalListener.getLeader())
+                                    .isEqualTo(leaderInformation);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testLeaderInformationChange() throws Exception {}
+    public void testLeaderInformationChangeNotifiesListener() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            final String componentA = "componentA";
+                            final LeaderInformation leaderInformationA =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            final String componentB = "componentB";
+                            final LeaderInformation leaderInformationB =
+                                    LeaderInformation.known(UUID.randomUUID(), "localhost");
+                            leaderElectionDriver.publishLeaderInformation(
+                                    componentA, leaderInformationA);
+                            leaderElectionDriver.publishLeaderInformation(
+                                    componentB, leaderInformationB);
+
+                            notifyLeaderElectionWatchOnModifiedConfigMap();
+
+                            final LeaderElectionEvent.AllKnownLeaderInformationEvent
+                                    allKnownLeaderInformationEvent =
+                                            leaderElectionListener.await(
+                                                    LeaderElectionEvent
+                                                            .AllKnownLeaderInformationEvent.class);
+
+                            assertThat(
+                                            allKnownLeaderInformationEvent
+                                                    .getLeaderInformationWithComponentIds())
+                                    .contains(

Review comment:
       `containsExactlyInAnyOrder` would be more precise here

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
##########
@@ -204,6 +204,23 @@
                     .withDescription(
                             "The time before a JobManager after a fail over recovers the current jobs.");
 
+    /**
+     * Safety hatch to fallback to the old ha services implementations.
+     *
+     * <p>Ideally, we can remove this option together with the old implementations in the next
+     * release.
+     */
+    @Documentation.Section(Documentation.Sections.EXPERT_HIGH_AVAILABILITY)
+    public static final ConfigOption<Boolean> USE_OLD_HA_SERVICES =
+            key("high-availability.use-old-ha-services")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "Use this option to disable the new HA service implementations for ZooKeeper and K8s. This is safety hatch in case that the new ha services are buggy.")

Review comment:
       nit: "[...] This is a safety hatch [...]"

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
##########
@@ -204,6 +204,23 @@
                     .withDescription(
                             "The time before a JobManager after a fail over recovers the current jobs.");
 
+    /**
+     * Safety hatch to fallback to the old ha services implementations.
+     *
+     * <p>Ideally, we can remove this option together with the old implementations in the next
+     * release.
+     */
+    @Documentation.Section(Documentation.Sections.EXPERT_HIGH_AVAILABILITY)

Review comment:
       FYI: Having the old implementation still in place makes life harder for other related issues like [FLINK-25432](https://issues.apache.org/jira/browse/FLINK-25432). But I guess, that's the save option. Could we create a follow-up ticket which we mention here?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesTestFixture.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+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.kubernetes.kubeclient.resources.KubernetesLeaderElector;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+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.CompletionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+
+/** Test fixture for Kubernetes tests that sets up a mock {@link FlinkKubeClient}. */
+class KubernetesTestFixture {
+    private static final long TIMEOUT = 30L * 1000L;
+
+    private final String leaderConfigmapName;
+    private final String lockIdentity;
+
+    private final Configuration configuration;
+
+    private final Map<String, KubernetesConfigMap> configMapStore = new HashMap<>();
+
+    private final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+            configMapCallbackFutures = new ArrayList<>();
+
+    private final List<TestingFlinkKubeClient.MockKubernetesWatch> configMapWatches =
+            new ArrayList<>();
+
+    private final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture =
+            new CompletableFuture<>();
+    private final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+    private final CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
+            leaderCallbackHandlerFuture = new CompletableFuture<>();
+
+    private final FlinkKubeClient flinkKubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+
+    KubernetesTestFixture(String clusterId, String leaderConfigmapName, String lockIdentity) {
+        this.leaderConfigmapName = leaderConfigmapName;
+        this.lockIdentity = lockIdentity;
+        configuration = new Configuration();
+        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
+
+        flinkKubeClient = createFlinkKubeClient();
+        configMapSharedWatcher =
+                flinkKubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+    }
+
+    void close() {
+        configMapSharedWatcher.close();
+    }
+
+    FlinkKubeClient getFlinkKubeClient() {
+        return flinkKubeClient;
+    }
+
+    CompletableFuture<Void> getCloseKubeClientFuture() {
+        return closeKubeClientFuture;
+    }
+
+    CompletableFuture<Map<String, String>> getDeleteConfigMapByLabelsFuture() {
+        return deleteConfigMapByLabelsFuture;
+    }
+
+    KubernetesConfigMapSharedWatcher getConfigMapSharedWatcher() {
+        return configMapSharedWatcher;
+    }
+
+    Configuration getConfiguration() {
+        return configuration;
+    }
+
+    KubernetesConfigMap getLeaderConfigMap() {
+        final Optional<KubernetesConfigMap> configMapOpt =
+                flinkKubeClient.getConfigMap(leaderConfigmapName);
+        assertThat(configMapOpt.isPresent(), is(true));
+        return configMapOpt.get();
+    }
+
+    // Use the leader callback to manually grant leadership
+    void leaderCallbackGrantLeadership() throws Exception {
+        createLeaderConfigMap();
+        getLeaderCallback().isLeader();
+    }
+
+    FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> getLeaderElectionConfigMapCallback()
+            throws Exception {
+        assertThat(configMapCallbackFutures.size(), is(greaterThanOrEqualTo(1)));

Review comment:
       We changed this assert from `=2` to `>=1`. That's done on purpose?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
##########
@@ -82,6 +80,31 @@ public static HighAvailabilityServices createAvailableOrEmbeddedServices(
         }
     }
 
+    @Nonnull

Review comment:
       Isn't the `@Nonnull` obsolete here again?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
##########
@@ -82,6 +80,31 @@ public static HighAvailabilityServices createAvailableOrEmbeddedServices(
         }
     }
 
+    @Nonnull
+    private static HighAvailabilityServices createZooKeeperHaServices(

Review comment:
       Why is there a `HighAvailabilityServicesFactory` implementation for k8s but not for ZooKeeper?

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -104,15 +102,6 @@ public void setup() throws Exception {
         savepointPath = temporaryFolder.newFolder("savepoints").getAbsolutePath();
     }
 
-    @After
-    public void teardown() throws Exception {
-        flinkKubeClient

Review comment:
       `flinkKubeClient` is not used anywhere anymore

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesTestFixture.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+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.kubernetes.kubeclient.resources.KubernetesLeaderElector;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+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.CompletionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+
+/** Test fixture for Kubernetes tests that sets up a mock {@link FlinkKubeClient}. */
+class KubernetesTestFixture {
+    private static final long TIMEOUT = 30L * 1000L;
+
+    private final String leaderConfigmapName;
+    private final String lockIdentity;
+
+    private final Configuration configuration;
+
+    private final Map<String, KubernetesConfigMap> configMapStore = new HashMap<>();
+
+    private final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+            configMapCallbackFutures = new ArrayList<>();
+
+    private final List<TestingFlinkKubeClient.MockKubernetesWatch> configMapWatches =
+            new ArrayList<>();
+
+    private final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture =
+            new CompletableFuture<>();
+    private final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+    private final CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
+            leaderCallbackHandlerFuture = new CompletableFuture<>();
+
+    private final FlinkKubeClient flinkKubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+
+    KubernetesTestFixture(String clusterId, String leaderConfigmapName, String lockIdentity) {
+        this.leaderConfigmapName = leaderConfigmapName;
+        this.lockIdentity = lockIdentity;
+        configuration = new Configuration();
+        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
+
+        flinkKubeClient = createFlinkKubeClient();
+        configMapSharedWatcher =
+                flinkKubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+    }
+
+    void close() {
+        configMapSharedWatcher.close();
+    }
+
+    FlinkKubeClient getFlinkKubeClient() {
+        return flinkKubeClient;
+    }
+
+    CompletableFuture<Void> getCloseKubeClientFuture() {
+        return closeKubeClientFuture;
+    }
+
+    CompletableFuture<Map<String, String>> getDeleteConfigMapByLabelsFuture() {
+        return deleteConfigMapByLabelsFuture;
+    }
+
+    KubernetesConfigMapSharedWatcher getConfigMapSharedWatcher() {
+        return configMapSharedWatcher;
+    }
+
+    Configuration getConfiguration() {
+        return configuration;
+    }
+
+    KubernetesConfigMap getLeaderConfigMap() {
+        final Optional<KubernetesConfigMap> configMapOpt =
+                flinkKubeClient.getConfigMap(leaderConfigmapName);
+        assertThat(configMapOpt.isPresent(), is(true));
+        return configMapOpt.get();
+    }
+
+    // Use the leader callback to manually grant leadership
+    void leaderCallbackGrantLeadership() throws Exception {
+        createLeaderConfigMap();
+        getLeaderCallback().isLeader();
+    }
+
+    FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> getLeaderElectionConfigMapCallback()
+            throws Exception {
+        assertThat(configMapCallbackFutures.size(), is(greaterThanOrEqualTo(1)));

Review comment:
       I'm also not 100% sure whether we're mixing up test code and assertions here in an unfavorable way? Reading the code I'm not certain why we do the assert here independent of any test method 🤔 I feel like we're assuming some context here that is not necessarily there.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {

Review comment:
       Correcting myself here: The `ZooKeeperMultipleComponentLeaderElectionDriver` implements `LeaderLatchListener` which provides the methods for losing and gaining leadership. The child nodes themselves don't lose leadership individually.
   
   I'm just wondering now why we need to check for `null` here? Isn't that a case that should never happen which means that we could add a `IllegalStateException` here? Or am I missing a use-case? ...in that case I'm wondering whether a log message would be valid at least.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java
##########
@@ -78,21 +84,46 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore(
             SharedStateRegistryFactory sharedStateRegistryFactory,
             Executor ioExecutor)
             throws Exception {
+        final String configMapName = getConfigMapNameFunction.apply(jobID);
+        KubernetesUtils.createConfigMapIfItDoesNotExist(kubeClient, configMapName, clusterId);
 
         return KubernetesUtils.createCompletedCheckpointStore(
                 configuration,
                 kubeClient,
                 executor,
-                getConfigMapNameFunction.apply(jobID),
+                configMapName,
                 lockIdentity,
                 maxNumberOfCheckpointsToRetain,
                 sharedStateRegistryFactory,
                 ioExecutor);
     }
 
     @Override
-    public CheckpointIDCounter createCheckpointIDCounter(JobID jobID) {
-        return new KubernetesCheckpointIDCounter(
-                kubeClient, getConfigMapNameFunction.apply(jobID), lockIdentity);
+    public CheckpointIDCounter createCheckpointIDCounter(JobID jobID) throws Exception {

Review comment:
       To me, it would feel more naturally to have a subclass of `KubernetesCheckpointRecoveryFactory` that takes care of the ConfigMap creation. But I don't have a strong argument towards refactoring this code because we wouldn't gain much from such a refactoring.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.Executors;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+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.TreeCache;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** ZooKeeper based {@link MultipleComponentLeaderElectionDriver} implementation. */
+public class ZooKeeperMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver, LeaderLatchListener {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ZooKeeperMultipleComponentLeaderElectionDriver.class);
+
+    private final CuratorFramework curatorFramework;
+
+    private final String leaderContenderDescription;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final LeaderLatch leaderLatch;
+
+    private final TreeCache treeCache;
+
+    private final ConnectionStateListener listener =
+            (client, newState) -> handleStateChange(newState);
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public ZooKeeperMultipleComponentLeaderElectionDriver(
+            CuratorFramework curatorFramework,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener)
+            throws Exception {
+        this.curatorFramework = curatorFramework;
+        this.leaderContenderDescription = leaderContenderDescription;
+        this.leaderElectionListener = leaderElectionListener;
+
+        this.leaderLatch = new LeaderLatch(curatorFramework, ZooKeeperUtils.getLeaderLatchNode());
+        this.treeCache =
+                TreeCache.newBuilder(curatorFramework, "/")
+                        .setCacheData(true)
+                        .setCreateParentNodes(false)
+                        .setSelector(
+                                new ZooKeeperMultipleComponentLeaderElectionDriver
+                                        .ConnectionInfoNodeSelector())
+                        .setExecutor(Executors.newDirectExecutorService())
+                        .build();
+        treeCache
+                .getListenable()
+                .addListener(
+                        (client, event) -> {
+                            switch (event.getType()) {
+                                case NODE_ADDED:
+                                case NODE_REMOVED:
+                                case NODE_UPDATED:
+                                    if (event.getData() != null) {
+                                        handleChangedLeaderInformation(event.getData());
+                                    }
+                            }
+                        });
+
+        leaderLatch.addListener(this);
+        curatorFramework.getConnectionStateListenable().addListener(listener);
+        leaderLatch.start();
+        treeCache.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            curatorFramework.getConnectionStateListenable().removeListener(listener);
+
+            Exception exception = null;
+
+            try {
+                treeCache.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            try {
+                leaderLatch.close();
+            } catch (Exception e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        return leaderLatch.hasLeadership();
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Write leader information {} for {}.", leaderInformation, componentId);
+        }
+        if (!leaderLatch.hasLeadership() || leaderInformation.isEmpty()) {
+            return;
+        }
+
+        final String connectionInformationPath =
+                ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+        ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                leaderInformation,
+                curatorFramework,
+                leaderLatch::hasLeadership,
+                connectionInformationPath);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String leaderName) throws Exception {
+        ZooKeeperUtils.deleteZNode(curatorFramework, ZooKeeperUtils.makeZooKeeperPath(leaderName));
+    }
+
+    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, waiting for reconnection.");
+                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 isLeader() {
+        LOG.debug("{} obtained the leadership.", this);
+        leaderElectionListener.isLeader();
+    }
+
+    @Override
+    public void notLeader() {
+        LOG.debug("{} lost the leadership.", this);
+        leaderElectionListener.notLeader();
+    }
+
+    private void handleChangedLeaderInformation(ChildData childData) {
+        if (running.get() && leaderLatch.hasLeadership() && isConnectionInfoNode(childData)) {
+
+            final String path = childData.getPath();
+            final String[] splits = path.split("/");

Review comment:
       same goes for extracting the `leaderName` below.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.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.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.KubernetesLeaderElectionConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.KubernetesConfigMapSharedWatcher;
+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.highavailability.AbstractHaServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.JobGraphStore;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+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.NAME_SEPARATOR;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Kubernetes HA services that use a single leader election service per JobManager. */
+public class KubernetesMultipleComponentLeaderElectionHaServices extends AbstractHaServices {
+
+    private final Object lock = new Object();
+
+    private final String clusterId;
+
+    private final FlinkKubeClient kubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+    private final ExecutorService watchExecutorService;
+
+    private final String lockIdentity;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private DefaultMultipleComponentLeaderElectionService multipleComponentLeaderElectionService =
+            null;
+
+    KubernetesMultipleComponentLeaderElectionHaServices(
+            FlinkKubeClient kubeClient,
+            Executor executor,
+            Configuration config,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler) {
+
+        super(config, executor, blobStoreService);
+        this.kubeClient = checkNotNull(kubeClient);
+        this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID));
+        this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+        this.configMapSharedWatcher =
+                this.kubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+        this.watchExecutorService =
+                Executors.newCachedThreadPool(
+                        new ExecutorThreadFactory("config-map-watch-handler"));
+
+        lockIdentity = UUID.randomUUID().toString();
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService =
+                getOrInitializeSingleLeaderElectionService();
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    private DefaultMultipleComponentLeaderElectionService
+            getOrInitializeSingleLeaderElectionService() {
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService == null) {
+                try {
+
+                    final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
+                            new KubernetesLeaderElectionConfiguration(
+                                    getClusterConfigMap(), lockIdentity, configuration);
+                    multipleComponentLeaderElectionService =
+                            new DefaultMultipleComponentLeaderElectionService(
+                                    fatalErrorHandler,
+                                    "Single leader election service",
+                                    new KubernetesMultipleComponentLeaderElectionDriverFactory(
+                                            kubeClient,
+                                            leaderElectionConfiguration,
+                                            configMapSharedWatcher,
+                                            watchExecutorService,
+                                            fatalErrorHandler));
+                } catch (Exception e) {
+                    throw new FlinkRuntimeException(
+                            "Could not initialize the default single leader election service.", e);
+                }
+            }
+
+            return multipleComponentLeaderElectionService;
+        }
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String componentId) {
+        return new DefaultLeaderRetrievalService(
+                new KubernetesMultipleComponentLeaderRetrievalDriverFactory(
+                        kubeClient,
+                        configMapSharedWatcher,
+                        watchExecutorService,
+                        getClusterConfigMap(),
+                        componentId));
+    }
+
+    @Override
+    protected CheckpointRecoveryFactory createCheckpointRecoveryFactory() {
+        return new KubernetesCheckpointRecoveryFactory(
+                kubeClient, configuration, ioExecutor, this::getJobSpecificConfigMap, lockIdentity);
+    }
+
+    private String getJobSpecificConfigMap(JobID jobID) {
+        return clusterId + NAME_SEPARATOR + jobID.toString() + NAME_SEPARATOR + "config-map";
+    }
+
+    @Override
+    protected JobGraphStore createJobGraphStore() throws Exception {
+        return KubernetesUtils.createJobGraphStore(
+                configuration, kubeClient, getClusterConfigMap(), lockIdentity);
+    }
+
+    private String getClusterConfigMap() {
+        return clusterId + NAME_SEPARATOR + "cluster-config-map";
+    }
+
+    @Override
+    protected RunningJobsRegistry createRunningJobsRegistry() {
+        return new KubernetesRunningJobsRegistry(kubeClient, getClusterConfigMap(), lockIdentity);
+    }
+
+    @Override
+    public void internalClose() throws Exception {
+        Exception exception = null;
+        try {
+            closeK8sServices();
+        } catch (Exception e) {
+            exception = e;
+        }
+
+        kubeClient.close();
+        ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, this.watchExecutorService);
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    private void closeK8sServices() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        configMapSharedWatcher.close();
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    public void internalCleanup() throws Exception {
+        Exception exception = null;
+        // in order to clean up, we first need to stop the services that rely on the config maps
+        try {
+            closeK8sServices();
+        } catch (Exception e) {
+            exception = e;
+        }
+
+        kubeClient
+                .deleteConfigMapsByLabels(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY))
+                .get();
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    public void internalCleanupJobData(JobID jobID) throws Exception {
+        kubeClient.deleteConfigMap(getJobSpecificConfigMap(jobID)).get();
+        // need to delete job specific leader address from leader config map
+    }
+
+    @Override
+    protected String getLeaderPathForResourceManager() {
+        return "resourcemanager";

Review comment:
       same goes for the other path-related methods below




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a605d8da083d456fbbae4c5adaaea270263b2f38 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperMultipleComponentLeaderElectionHaServices.java
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.zookeeper;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStoreService;
+import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.ZooKeeperMultipleComponentLeaderElectionDriverFactory;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.concurrent.Executor;
+
+/**
+ * ZooKeeper HA services that only use a single leader election per process.
+ *
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/leader/latch
+ *      |            |       /resource_manager/connection_info
+ *      |            |       /dispatcher/connection_info
+ *      |            |       /rest_server/connection_info
+ *      |            |       /job-id-1/connection_info
+ *      |            |       /job-id-2/connection_info
+ *      |            |
+ *      |            |
+ *      |            +jobgraphs/job-id-1
+ *      |            |         /job-id-2
+ *      |            +jobs/job-id-1/checkpoints/latest
+ *      |                 |                    /latest-1
+ *      |                 |                    /latest-2
+ *      |                 |       /checkpoint_id_counter
+ * </pre>
+ */
+public class ZooKeeperMultipleComponentLeaderElectionHaServices
+        extends AbstractZooKeeperHaServices {
+
+    private final Object lock = new Object();
+
+    private final CuratorFramework leaderNamespacedCuratorFramework;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @Nullable
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService multipleComponentLeaderElectionService = null;
+
+    public ZooKeeperMultipleComponentLeaderElectionHaServices(
+            CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper,
+            Configuration config,
+            Executor ioExecutor,
+            BlobStoreService blobStoreService,
+            FatalErrorHandler fatalErrorHandler)
+            throws Exception {
+        super(curatorFrameworkWrapper, ioExecutor, config, blobStoreService);
+        this.leaderNamespacedCuratorFramework =
+                ZooKeeperUtils.useNamespaceAndEnsurePath(
+                        getCuratorFramework(), ZooKeeperUtils.getLeaderPath());
+        this.fatalErrorHandler = fatalErrorHandler;
+    }
+
+    @Override
+    protected LeaderElectionService createLeaderElectionService(String leaderName) {
+        final MultipleComponentLeaderElectionService multipleComponentLeaderElectionService;
+
+        synchronized (lock) {
+            multipleComponentLeaderElectionService = getOrInitializeSingleLeaderElectionService();
+        }
+
+        return new DefaultLeaderElectionService(
+                multipleComponentLeaderElectionService.createDriverFactory(leaderName));
+    }
+
+    @GuardedBy("lock")
+    private MultipleComponentLeaderElectionService getOrInitializeSingleLeaderElectionService() {
+        if (multipleComponentLeaderElectionService == null) {
+            try {
+                multipleComponentLeaderElectionService =
+                        new DefaultMultipleComponentLeaderElectionService(
+                                fatalErrorHandler,
+                                "Single leader election service.",
+                                new ZooKeeperMultipleComponentLeaderElectionDriverFactory(
+                                        leaderNamespacedCuratorFramework));
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Could not initialize the %s",
+                                DefaultMultipleComponentLeaderElectionService.class
+                                        .getSimpleName()),
+                        e);
+            }
+        }
+
+        return multipleComponentLeaderElectionService;
+    }
+
+    @Override
+    protected LeaderRetrievalService createLeaderRetrievalService(String leaderPath) {
+        // Maybe use a single service for leader retrieval
+        return ZooKeeperUtils.createLeaderRetrievalService(
+                leaderNamespacedCuratorFramework, leaderPath, configuration);
+    }
+
+    @Override
+    protected void internalClose() throws Exception {
+        Exception exception = null;
+        synchronized (lock) {
+            if (multipleComponentLeaderElectionService != null) {
+                try {
+                    multipleComponentLeaderElectionService.close();
+                } catch (Exception e) {
+                    exception = e;
+                }
+                multipleComponentLeaderElectionService = null;
+            }
+        }
+
+        try {
+            super.internalClose();
+        } catch (Exception e) {
+            exception = ExceptionUtils.firstOrSuppressed(e, exception);
+        }
+
+        ExceptionUtils.tryRethrowException(exception);
+    }
+
+    @Override
+    protected void internalCleanupJobData(JobID jobID) throws Exception {
+        super.internalCleanupJobData(jobID);
+    }
+
+    @Override
+    protected String getLeaderPathForResourceManager() {
+        return ZooKeeperUtils.getResourceManagerNode();
+    }
+
+    @Override
+    protected String getLeaderPathForDispatcher() {
+        return ZooKeeperUtils.getDispatcherNode();
+    }
+
+    @Override
+    protected String getLeaderPathForJobManager(JobID jobID) {
+        return jobID.toString();
+    }
+
+    @Override
+    protected String getLeaderPathForRestServer() {
+        return ZooKeeperUtils.getRestServerNode();
+    }

Review comment:
       Arg, the problem is that the different `AbstractHaServices` implementations treat this method differently. Since I don't want to touch the `ZooKeeperHaServices`, I will keep it as is. Once `ZooKeeperHaServices` is removed, we can clean it up.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;
+
+    @Nullable
+    @GuardedBy("lock")
+    private UUID currentLeaderSessionId = null;
+
+    @VisibleForTesting
+    DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory,
+            ExecutorService leadershipOperationExecutor)
+            throws Exception {
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.leadershipOperationExecutor = leadershipOperationExecutor;
+
+        leaderElectionEventHandlers = new HashMap<>();
+
+        multipleComponentLeaderElectionDriver =
+                multipleComponentLeaderElectionDriverFactory.create(
+                        leaderContenderDescription, this);
+    }
+
+    public DefaultMultipleComponentLeaderElectionService(
+            FatalErrorHandler fatalErrorHandler,
+            String leaderContenderDescription,
+            MultipleComponentLeaderElectionDriverFactory
+                    multipleComponentLeaderElectionDriverFactory)
+            throws Exception {
+        this(
+                fatalErrorHandler,
+                leaderContenderDescription,
+                multipleComponentLeaderElectionDriverFactory,
+                java.util.concurrent.Executors.newSingleThreadExecutor(
+                        new ExecutorThreadFactory(
+                                String.format(
+                                        "leadershipOperation-%s", leaderContenderDescription))));
+    }
+
+    @Override
+    public void close() throws Exception {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+            running = false;
+
+            LOG.info("Closing {}.", this);
+
+            ExecutorUtils.gracefulShutdown(10L, TimeUnit.SECONDS, leadershipOperationExecutor);
+
+            Exception exception = null;
+            try {
+                multipleComponentLeaderElectionDriver.close();
+            } catch (Exception e) {
+                exception = e;
+            }
+
+            ExceptionUtils.tryRethrowException(exception);
+        }
+    }
+
+    @Override
+    public LeaderElectionDriverFactory createDriverFactory(String leaderName) {
+        return new MultipleComponentLeaderElectionDriverAdapterFactory(leaderName, this);
+    }
+
+    @Override
+    public void publishLeaderInformation(String leaderName, LeaderInformation leaderInformation) {
+        try {
+            multipleComponentLeaderElectionDriver.publishLeaderInformation(
+                    leaderName, leaderInformation);
+        } catch (Exception e) {
+            fatalErrorHandler.onFatalError(
+                    new FlinkException(
+                            String.format(
+                                    "Could not write leader information %s for leader %s.",
+                                    leaderInformation, leaderName),
+                            e));
+        }
+    }
+
+    @Override
+    public void registerLeaderElectionEventHandler(
+            String componentId, LeaderElectionEventHandler leaderElectionEventHandler) {
+
+        synchronized (lock) {
+            Preconditions.checkState(
+                    !leaderElectionEventHandlers.containsKey(componentId),
+                    "Do not support duplicate LeaderElectionEventHandler registration under %s",
+                    componentId);
+            leaderElectionEventHandlers.put(componentId, leaderElectionEventHandler);
+
+            if (currentLeaderSessionId != null) {
+                leadershipOperationExecutor.execute(
+                        () -> leaderElectionEventHandler.onGrantLeadership(currentLeaderSessionId));
+            }
+        }
+    }
+
+    @Override
+    public void unregisterLeaderElectionEventHandler(String componentId) throws Exception {
+        final LeaderElectionEventHandler unregisteredLeaderElectionEventHandler;
+        synchronized (lock) {
+            unregisteredLeaderElectionEventHandler =
+                    leaderElectionEventHandlers.remove(componentId);
+
+            if (unregisteredLeaderElectionEventHandler != null) {
+                leadershipOperationExecutor.execute(
+                        unregisteredLeaderElectionEventHandler::onRevokeLeadership);
+            }
+        }
+
+        multipleComponentLeaderElectionDriver.deleteLeaderInformation(componentId);
+    }
+
+    @Override
+    public boolean hasLeadership(String leaderName) {
+        synchronized (lock) {
+            Preconditions.checkState(running);
+
+            return leaderElectionEventHandlers.containsKey(leaderName)
+                    && multipleComponentLeaderElectionDriver.hasLeadership();
+        }
+    }
+
+    @Override
+    public void isLeader() {
+        final UUID newLeaderSessionId = UUID.randomUUID();
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            currentLeaderSessionId = UUID.randomUUID();
+
+            forEachLeaderElectionEventHandler(
+                    leaderElectionEventHandler ->
+                            leaderElectionEventHandler.onGrantLeadership(newLeaderSessionId));
+        }
+    }
+
+    @Override
+    public void notLeader() {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            currentLeaderSessionId = null;
+
+            forEachLeaderElectionEventHandler(LeaderElectionEventHandler::onRevokeLeadership);
+        }
+    }
+
+    @GuardedBy("lock")
+    private void forEachLeaderElectionEventHandler(
+            Consumer<? super LeaderElectionEventHandler> action) {
+
+        for (LeaderElectionEventHandler leaderElectionEventHandler :
+                leaderElectionEventHandlers.values()) {
+            leadershipOperationExecutor.execute(() -> action.accept(leaderElectionEventHandler));
+        }
+    }
+
+    @Override
+    public void notifyLeaderInformationChange(
+            String leaderName, LeaderInformation leaderInformation) {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            final LeaderElectionEventHandler leaderElectionEventHandler =
+                    leaderElectionEventHandlers.get(leaderName);
+
+            if (leaderElectionEventHandler != null) {
+                leadershipOperationExecutor.execute(
+                        () ->
+                                leaderElectionEventHandler.onLeaderInformationChange(
+                                        leaderInformation));
+            }
+        }
+    }
+
+    @Override
+    public void notifyAllKnownLeaderInformation(
+            Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {
+        synchronized (lock) {
+            if (!running) {
+                return;
+            }
+
+            final Map<String, LeaderInformation> leaderInformationByName =
+                    leaderInformationWithComponentIds.stream()
+                            .collect(
+                                    Collectors.toMap(
+                                            LeaderInformationWithComponentId::getComponentId,
+                                            LeaderInformationWithComponentId
+                                                    ::getLeaderInformation));
+
+            for (Map.Entry<String, LeaderElectionEventHandler>
+                    leaderNameLeaderElectionEventHandlerPair :
+                            leaderElectionEventHandlers.entrySet()) {
+                final String leaderName = leaderNameLeaderElectionEventHandlerPair.getKey();
+                if (leaderInformationByName.containsKey(leaderName)) {
+                    leaderNameLeaderElectionEventHandlerPair
+                            .getValue()
+                            .onLeaderInformationChange(leaderInformationByName.get(leaderName));

Review comment:
       I think we filter out these cases on a higher level.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultMultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of a {@link MultipleComponentLeaderElectionService} that allows to
+ * register multiple {@link LeaderElectionEventHandler}.
+ */
+public class DefaultMultipleComponentLeaderElectionService
+        implements MultipleComponentLeaderElectionService,
+                MultipleComponentLeaderElectionDriver.Listener {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultMultipleComponentLeaderElectionService.class);
+
+    private final Object lock = new Object();
+
+    private final MultipleComponentLeaderElectionDriver multipleComponentLeaderElectionDriver;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    @GuardedBy("lock")
+    private final ExecutorService leadershipOperationExecutor;
+
+    @GuardedBy("lock")
+    private final Map<String, LeaderElectionEventHandler> leaderElectionEventHandlers;
+
+    private boolean running = true;

Review comment:
       Good catch. Will change 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -96,13 +106,17 @@
     /** The prefix of the completed checkpoint file. */
     public static final String HA_STORAGE_COMPLETED_CHECKPOINT = "completedCheckpoint";
 
-    private static final String RESOURCE_MANAGER_LEADER = "/resource_manager";
+    private static final String RESOURCE_MANAGER_LEADER = "resource_manager";
 
-    private static final String DISPATCHER_LEADER = "/dispatcher";
+    private static final String DISPATCHER_LEADER = "dispatcher";
 
     private static final String LEADER_NODE = "/leader";

Review comment:
       True. This is a nice unification. Will apply 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/MultipleComponentLeaderElectionService.java
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+/**
+ * Leader election service that allows to register multiple {@link LeaderElectionEventHandler
+ * LeaderElectionEventHandlers} that are identified by different names. For each event handler it is
+ * possible to write the corresponding {@link LeaderInformation}.
+ */
+public interface MultipleComponentLeaderElectionService {

Review comment:
       This is probably a good idea. Will add 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -137,17 +155,37 @@ public static String getCheckpointIdCounterPath() {
         return "/checkpoint_id_counter";
     }
 
+    public static String getLeaderPath() {
+        return LEADER_NODE;
+    }
+
+    public static String getDispatcherNode() {
+        return DISPATCHER_LEADER;
+    }
+
+    public static String getResourceManagerNode() {
+        return RESOURCE_MANAGER_LEADER;
+    }
+
+    public static String getRestServerNode() {
+        return REST_SERVER_LEADER;
+    }
+
+    public static String getLeaderLatchNode() {
+        return LEADER_LATCH_NODE;
+    }
+
     private static String getLeaderPath(String suffix) {
         return generateZookeeperPath(LEADER_NODE, suffix);
     }
 
     @Nonnull

Review comment:
       True, but I haven't touched this code in this PR. 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";
+            final DefaultLeaderRetrievalService defaultLeaderRetrievalService =
+                    new DefaultLeaderRetrievalService(
+                            new ZooKeeperLeaderRetrievalDriverFactory(
+                                    curatorFramework.asCuratorFramework(),
+                                    componentId,
+                                    ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                            .ON_LOST_CONNECTION));
+
+            final TestingListener leaderRetrievalListener = new TestingListener();
+            defaultLeaderRetrievalService.start(leaderRetrievalListener);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            leaderElectionDriver.publishLeaderInformation(componentId, leaderInformation);
+
+            leaderRetrievalListener.waitForNewLeader(10_000L);
+
+            assertThat(leaderRetrievalListener.getLeader()).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderInformationChange() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final LeaderInformation leaderInformation =
+                    LeaderInformation.known(UUID.randomUUID(), "foobar");
+            final String componentId = "barfoo";
+            final String path = ZooKeeperUtils.generateConnectionInformationPath(componentId);
+
+            ZooKeeperUtils.writeLeaderInformationToZooKeeper(
+                    leaderInformation, curatorFramework.asCuratorFramework(), () -> true, path);
+
+            final LeaderInformationChangeEvent leaderInformationChangeEvent =
+                    leaderElectionListener.await(LeaderInformationChangeEvent.class);
+
+            assertThat(leaderInformationChangeEvent.componentId).isEqualTo(componentId);
+            assertThat(leaderInformationChangeEvent.leaderInformation).isEqualTo(leaderInformation);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testLeaderElectionWithMultipleDrivers() throws Exception {
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        try {
+            Set<ElectionDriver> electionDrivers =
+                    Stream.generate(
+                                    () ->
+                                            createLeaderElectionDriver(
+                                                    curatorFramework.asCuratorFramework()))
+                            .limit(3)
+                            .collect(Collectors.toSet());
+
+            while (!electionDrivers.isEmpty()) {
+                final CompletableFuture<Object> anyLeader =
+                        CompletableFuture.anyOf(
+                                electionDrivers.stream()
+                                        .map(ElectionDriver::getLeadershipFuture)
+                                        .collect(Collectors.toList())
+                                        .toArray(new CompletableFuture[0]));
+
+                // wait for any leader
+                anyLeader.join();
+
+                final Map<Boolean, Set<ElectionDriver>> leaderAndRest =
+                        electionDrivers.stream()
+                                .collect(
+                                        Collectors.partitioningBy(
+                                                ElectionDriver::hasLeadership, Collectors.toSet()));
+
+                assertThat(leaderAndRest.get(true)).hasSize(1);
+                Iterables.getOnlyElement(leaderAndRest.get(true)).close();
+
+                electionDrivers = leaderAndRest.get(false);
+            }
+        } finally {
+            curatorFramework.close();
+        }
+    }
+
+    private static ElectionDriver createLeaderElectionDriver(CuratorFramework curatorFramework) {
+        final SimpleLeaderElectionListener leaderElectionListener =
+                new SimpleLeaderElectionListener();
+
+        try {
+            final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                    startLeaderElectionDriver(leaderElectionListener, curatorFramework);
+            return new ElectionDriver(leaderElectionDriver, leaderElectionListener);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+            return null;
+        }
+    }
+
+    private static final class ElectionDriver {
+        private final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver;
+        private final SimpleLeaderElectionListener leaderElectionListener;
+
+        private ElectionDriver(
+                ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver,
+                SimpleLeaderElectionListener leaderElectionListener) {
+            this.leaderElectionDriver = leaderElectionDriver;
+            this.leaderElectionListener = leaderElectionListener;
+        }
+
+        void close() throws Exception {
+            leaderElectionDriver.close();
+        }
+
+        boolean hasLeadership() {
+            return leaderElectionDriver.hasLeadership();
+        }
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leaderElectionListener.getLeadershipFuture();
+        }
+    }
+
+    private static final class SimpleLeaderElectionListener
+            implements MultipleComponentLeaderElectionDriver.Listener {
+
+        private final CompletableFuture<Void> leadershipFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> getLeadershipFuture() {
+            return leadershipFuture;
+        }
+
+        @Override
+        public void isLeader() {
+            leadershipFuture.complete(null);
+        }
+
+        @Override
+        public void notLeader() {}
+
+        @Override
+        public void notifyLeaderInformationChange(
+                String componentId, LeaderInformation leaderInformation) {}
+
+        @Override
+        public void notifyAllKnownLeaderInformation(
+                Collection<LeaderInformationWithComponentId> leaderInformationWithComponentIds) {}
+    }
+
+    @Nonnull
+    private static ZooKeeperMultipleComponentLeaderElectionDriver startLeaderElectionDriver(
+            MultipleComponentLeaderElectionDriver.Listener leaderElectionListener,
+            CuratorFramework curatorFramework)
+            throws Exception {
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                new ZooKeeperMultipleComponentLeaderElectionDriver(
+                        curatorFramework, "foobar", leaderElectionListener);
+        return leaderElectionDriver;
+    }
+
+    @Nonnull

Review comment:
       Will remove 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
+import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriverFactory;
+import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
+import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
+@ExtendWith(TestLoggerExtension.class)
+class ZooKeeperMultipleComponentLeaderElectionDriverTest {
+
+    private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension();
+
+    @RegisterExtension
+    private final EachCallbackWrapper<ZooKeeperExtension> eachWrapper =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @Test
+    public void testElectionDriverGainsLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testElectionDriverLosesLeadership() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            zooKeeperExtension.stop();
+
+            leaderElectionListener.await(NotLeaderEvent.class);
+        } finally {
+            leaderElectionDriver.close();
+            curatorFramework.close();
+        }
+    }
+
+    @Test
+    public void testPublishLeaderInformation() throws Exception {
+        final TestingLeaderElectionListener leaderElectionListener =
+                new TestingLeaderElectionListener();
+        final CuratorFrameworkWithUnhandledErrorListener curatorFramework = startCuratorFramework();
+
+        final ZooKeeperMultipleComponentLeaderElectionDriver leaderElectionDriver =
+                startLeaderElectionDriver(
+                        leaderElectionListener, curatorFramework.asCuratorFramework());
+
+        try {
+            leaderElectionListener.await(IsLeaderEvent.class);
+
+            final String componentId = "barfoo";

Review comment:
       Will change 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriver.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher;
+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.utils.KubernetesUtils;
+import org.apache.flink.runtime.leaderelection.LeaderElectionException;
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.LeaderInformationWithComponentId;
+import org.apache.flink.runtime.leaderelection.MultipleComponentLeaderElectionDriver;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.apache.flink.kubernetes.utils.KubernetesUtils.checkConfigMaps;
+
+/** {@link MultipleComponentLeaderElectionDriver} for Kubernetes. */
+public class KubernetesMultipleComponentLeaderElectionDriver
+        implements MultipleComponentLeaderElectionDriver {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(KubernetesMultipleComponentLeaderElectionDriver.class);
+
+    private final FlinkKubeClient kubeClient;
+
+    private final String configMapName;
+
+    private final String lockIdentity;
+
+    private final MultipleComponentLeaderElectionDriver.Listener leaderElectionListener;
+
+    private final KubernetesLeaderElector leaderElector;
+
+    // Labels will be used to clean up the ha related ConfigMaps.
+    private final Map<String, String> configMapLabels;
+
+    private final FatalErrorHandler fatalErrorHandler;
+
+    private final KubernetesSharedWatcher.Watch kubernetesWatch;
+
+    private AtomicBoolean running = new AtomicBoolean(true);
+
+    public KubernetesMultipleComponentLeaderElectionDriver(
+            KubernetesLeaderElectionConfiguration leaderElectionConfiguration,
+            FlinkKubeClient kubeClient,
+            Listener leaderElectionListener,
+            KubernetesConfigMapSharedWatcher configMapSharedWatcher,
+            Executor watchExecutor,
+            FatalErrorHandler fatalErrorHandler) {
+        this.kubeClient = kubeClient;
+        this.leaderElectionListener = leaderElectionListener;
+        this.fatalErrorHandler = fatalErrorHandler;
+
+        this.configMapName = leaderElectionConfiguration.getConfigMapName();
+        this.lockIdentity = leaderElectionConfiguration.getLockIdentity();
+
+        this.leaderElector =
+                kubeClient.createLeaderElector(
+                        leaderElectionConfiguration, new LeaderCallbackHandlerImpl());
+
+        this.configMapLabels =
+                KubernetesUtils.getConfigMapLabels(
+                        leaderElectionConfiguration.getClusterId(),
+                        LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY);
+
+        kubernetesWatch =
+                configMapSharedWatcher.watch(
+                        configMapName, new ConfigMapCallbackHandlerImpl(), watchExecutor);
+
+        leaderElector.run();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (running.compareAndSet(true, false)) {
+            LOG.info("Closing {}.", this);
+
+            leaderElector.stop();
+            kubernetesWatch.close();
+        }
+    }
+
+    @Override
+    public boolean hasLeadership() {
+        Preconditions.checkState(running.get());
+        final Optional<KubernetesConfigMap> optionalConfigMap =
+                kubeClient.getConfigMap(configMapName);
+
+        if (optionalConfigMap.isPresent()) {
+            return KubernetesLeaderElector.hasLeadership(optionalConfigMap.get(), lockIdentity);
+        } else {
+            fatalErrorHandler.onFatalError(
+                    new KubernetesException(
+                            String.format(
+                                    "ConfigMap %s does not exist. This indicates that somebody has interfered with Flink's operation.",
+                                    configMapName)));
+            return false;
+        }
+    }
+
+    @Override
+    public void publishLeaderInformation(String componentId, LeaderInformation leaderInformation)
+            throws Exception {
+        Preconditions.checkState(running.get());
+
+        kubeClient
+                .checkAndUpdateConfigMap(
+                        configMapName,
+                        updateConfigMapWithLeaderInformation(componentId, leaderInformation))
+                .get();
+
+        LOG.debug(
+                "Successfully wrote leader information {} for leader {} into the config map {}.",
+                leaderInformation,
+                componentId,
+                configMapName);
+    }
+
+    @Override
+    public void deleteLeaderInformation(String componentId) throws Exception {
+        publishLeaderInformation(componentId, LeaderInformation.empty());
+    }
+
+    private Function<KubernetesConfigMap, Optional<KubernetesConfigMap>>
+            updateConfigMapWithLeaderInformation(
+                    String leaderName, LeaderInformation leaderInformation) {
+        final String configMapDataKey = KubernetesUtils.createSingleLeaderKey(leaderName);
+
+        return kubernetesConfigMap -> {
+            if (KubernetesLeaderElector.hasLeadership(kubernetesConfigMap, lockIdentity)) {
+                final Map<String, String> data = kubernetesConfigMap.getData();
+
+                if (leaderInformation.isEmpty()) {
+                    data.remove(configMapDataKey);
+                } else {
+                    data.put(
+                            configMapDataKey,
+                            KubernetesUtils.encodeLeaderInformation(leaderInformation));
+                }
+
+                kubernetesConfigMap.getLabels().putAll(configMapLabels);
+                return Optional.of(kubernetesConfigMap);
+            }
+
+            return Optional.empty();
+        };
+    }
+
+    private static Collection<LeaderInformationWithComponentId> extractLeaderInformation(
+            KubernetesConfigMap configMap) {
+        final Map<String, String> data = configMap.getData();
+
+        final Collection<LeaderInformationWithComponentId> leaderInformationWithLeaderNames =
+                new ArrayList<>();
+
+        for (Map.Entry<String, String> keyValuePair : data.entrySet()) {
+            final String key = keyValuePair.getKey();
+            if (KubernetesUtils.isSingleLeaderKey(key)) {
+                final String leaderName = KubernetesUtils.extractLeaderName(key);
+                final LeaderInformation leaderInformation =
+                        KubernetesUtils.parseLeaderInformationSafely(keyValuePair.getValue());

Review comment:
       I think this is ok because the corresponding `DefaultLeaderElectionService` would issue another publish command if it is the leader and its leader information is 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesTestFixture.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.KubernetesConfigMapSharedWatcher;
+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.kubernetes.kubeclient.resources.KubernetesLeaderElector;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+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.CompletionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector.LEADER_ANNOTATION_KEY;
+import static org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+
+/** Test fixture for Kubernetes tests that sets up a mock {@link FlinkKubeClient}. */
+class KubernetesTestFixture {
+    private static final long TIMEOUT = 30L * 1000L;
+
+    private final String leaderConfigmapName;
+    private final String lockIdentity;
+
+    private final Configuration configuration;
+
+    private final Map<String, KubernetesConfigMap> configMapStore = new HashMap<>();
+
+    private final List<CompletableFuture<FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap>>>
+            configMapCallbackFutures = new ArrayList<>();
+
+    private final List<TestingFlinkKubeClient.MockKubernetesWatch> configMapWatches =
+            new ArrayList<>();
+
+    private final CompletableFuture<Map<String, String>> deleteConfigMapByLabelsFuture =
+            new CompletableFuture<>();
+    private final CompletableFuture<Void> closeKubeClientFuture = new CompletableFuture<>();
+
+    private final CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
+            leaderCallbackHandlerFuture = new CompletableFuture<>();
+
+    private final FlinkKubeClient flinkKubeClient;
+
+    private final KubernetesConfigMapSharedWatcher configMapSharedWatcher;
+
+    KubernetesTestFixture(String clusterId, String leaderConfigmapName, String lockIdentity) {
+        this.leaderConfigmapName = leaderConfigmapName;
+        this.lockIdentity = lockIdentity;
+        configuration = new Configuration();
+        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
+
+        flinkKubeClient = createFlinkKubeClient();
+        configMapSharedWatcher =
+                flinkKubeClient.createConfigMapSharedWatcher(
+                        KubernetesUtils.getConfigMapLabels(
+                                clusterId, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
+    }
+
+    void close() {
+        configMapSharedWatcher.close();
+    }
+
+    FlinkKubeClient getFlinkKubeClient() {
+        return flinkKubeClient;
+    }
+
+    CompletableFuture<Void> getCloseKubeClientFuture() {
+        return closeKubeClientFuture;
+    }
+
+    CompletableFuture<Map<String, String>> getDeleteConfigMapByLabelsFuture() {
+        return deleteConfigMapByLabelsFuture;
+    }
+
+    KubernetesConfigMapSharedWatcher getConfigMapSharedWatcher() {
+        return configMapSharedWatcher;
+    }
+
+    Configuration getConfiguration() {
+        return configuration;
+    }
+
+    KubernetesConfigMap getLeaderConfigMap() {
+        final Optional<KubernetesConfigMap> configMapOpt =
+                flinkKubeClient.getConfigMap(leaderConfigmapName);
+        assertThat(configMapOpt.isPresent(), is(true));
+        return configMapOpt.get();
+    }
+
+    // Use the leader callback to manually grant leadership
+    void leaderCallbackGrantLeadership() throws Exception {
+        createLeaderConfigMap();
+        getLeaderCallback().isLeader();
+    }
+
+    FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> getLeaderElectionConfigMapCallback()
+            throws Exception {
+        assertThat(configMapCallbackFutures.size(), is(greaterThanOrEqualTo(1)));

Review comment:
       We probably are. I am also not super happy with the current state but I also didn't want to completely refactor the existing K8s tests. Hence, I would suggest to create a follow up task for cleaning this up.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java
##########
@@ -56,59 +62,179 @@
             new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor);
 
     @Test
-    public void testElectionDriverGainsLeadership() throws InterruptedException {
-        final Configuration configuration = new Configuration();
-        configuration.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID);
-        final KubernetesLeaderElectionConfiguration leaderElectionConfiguration =
-                new KubernetesLeaderElectionConfiguration("foobar", "barfoo", configuration);
-
-        CompletableFuture<KubernetesLeaderElector.LeaderCallbackHandler>
-                leaderCallbackHandlerFuture = new CompletableFuture<>();
-        final FlinkKubeClient flinkKubeClient =
-                TestingFlinkKubeClient.builder()
-                        .setCreateLeaderElectorFunction(
-                                (leaderConfig, callbackHandler) -> {
-                                    leaderCallbackHandlerFuture.complete(callbackHandler);
-                                    return new TestingFlinkKubeClient
-                                            .TestingKubernetesLeaderElector(
-                                            leaderConfig, callbackHandler);
-                                })
-                        .build();
-
-        final KubernetesConfigMapSharedWatcher configMapSharedWatcher =
-                flinkKubeClient.createConfigMapSharedWatcher(
-                        KubernetesUtils.getConfigMapLabels(
-                                CLUSTER_ID, LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY));
-
-        final TestingLeaderElectionListener leaderElectionListener =
-                new TestingLeaderElectionListener();
-
-        final KubernetesMultipleComponentLeaderElectionDriver leaderElectionDriver =
-                new KubernetesMultipleComponentLeaderElectionDriver(
-                        leaderElectionConfiguration,
-                        flinkKubeClient,
-                        leaderElectionListener,
-                        configMapSharedWatcher,
-                        testExecutorExtension.getExecutor(),
-                        testingFatalErrorHandlerExtension.getTestingFatalErrorHandler());
-
-        final KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler =
-                leaderCallbackHandlerFuture.join();
-
-        leaderCallbackHandler.isLeader();
-
-        leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+    public void testElectionDriverGainsLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                        });
+            }
+        };
     }
 
     @Test
-    public void testElectionDriverLosesLeadership() throws Exception {}
+    public void testElectionDriverLosesLeadership() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();
+                            leaderElectionListener.await(LeaderElectionEvent.IsLeaderEvent.class);
+                            getLeaderCallback().notLeader();
+                            leaderElectionListener.await(LeaderElectionEvent.NotLeaderEvent.class);
+                        });
+            }
+        };
+    }
 
     @Test
-    public void testPublishLeaderInformation() throws Exception {}
+    public void testPublishLeaderInformation() throws Exception {
+        new TestFixture() {
+            {
+                runTest(
+                        () -> {
+                            leaderCallbackGrantLeadership();

Review comment:
       This is necessary to give leadership to the contender.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30248",
       "triggerID" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7aaf594b9767994a95d9cba1c2f7e60d385002a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30248) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -248,5 +229,24 @@ public void run(SourceContext<Integer> ctx) throws Exception {
         public void cancel() {
             running = false;
         }
+
+        @Override
+        public void snapshotState(FunctionSnapshotContext context) throws Exception {}
+
+        @Override
+        public void initializeState(FunctionInitializationContext context) throws Exception {
+            final ListState<Integer> hasExecutedBeforeState =
+                    context.getOperatorStateStore()
+                            .getUnionListState(hasExecutedBeforeStateDescriptor);
+
+            // if we have state, then we resume from a savepoint --> stop the execution then
+            if (hasExecutedBeforeState.get().iterator().hasNext()) {
+                running = false;
+            }
+
+            hasExecutedBeforeState.clear();

Review comment:
       It is to have a clean state into which we insert the subtask index.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] tillrohrmann commented on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   Thanks for the review @XComp. I've resolved most of your comments. Those that I haven't resolved, I added an explanation for why not. Please take another look at this PR so that we can finish it swiftly :-)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #17485: [FLINK-24038] Add support for single leader election per JobManager process

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25056",
       "triggerID" : "6934d6236ab71309c59fab4432aa3234fe916e12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29437",
       "triggerID" : "00e0b2324b8c72cf98105f985d40c006a43d6ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29473",
       "triggerID" : "a605d8da083d456fbbae4c5adaaea270263b2f38",
       "triggerType" : "PUSH"
     }, {
       "hash" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29487",
       "triggerID" : "972895e90d96516a2b8fa3722daf677e7759e266",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29489",
       "triggerID" : "f8f21fd77139eee5f97f39195d31fcb67ae6d996",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29503",
       "triggerID" : "dd4a85dae4b59ff533c3cfdba8bfc28dae5e2ebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30155",
       "triggerID" : "a3c818cb7a4341243bbf7f584d327a1f51d355d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30179",
       "triggerID" : "9bd3feac7400f1ad0f6e14c1d37308cc9ff95011",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245",
       "triggerID" : "3940b1c8714c3a58d3e6a036a4ee9669b282c1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30248",
       "triggerID" : "c7aaf594b9767994a95d9cba1c2f7e60d385002a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3940b1c8714c3a58d3e6a036a4ee9669b282c1bc Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30245) 
   * c7aaf594b9767994a95d9cba1c2f7e60d385002a Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30248) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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