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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to