wangyang0918 commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r513904491



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionDriver.java
##########
@@ -0,0 +1,314 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.ExceptionUtils;
+
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
+import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link LeaderElectionDriver} implementation for Zookeeper. The leading 
JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is 
published via
+ * ZooKeeper.
+ */
+public class ZooKeeperLeaderElectionDriver implements LeaderElectionDriver, 
LeaderLatchListener, NodeCacheListener, UnhandledErrorListener {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(ZooKeeperLeaderElectionDriver.class);
+
+       private final Object lock = new Object();
+
+       /** Client to the ZooKeeper quorum. */
+       private final CuratorFramework client;
+
+       /** Curator recipe for leader election. */
+       private final LeaderLatch leaderLatch;
+
+       /** Curator recipe to watch a given ZooKeeper node for changes. */
+       private final NodeCache cache;
+
+       /** ZooKeeper path of the node which stores the current leader 
information. */
+       private final String leaderPath;
+
+       private final ConnectionStateListener listener = (client, newState) -> 
handleStateChange(newState);
+
+       private final LeaderElectionEventHandler leaderElectionEventHandler;
+
+       private final FatalErrorHandler fatalErrorHandler;
+
+       private final String leaderContenderDescription;
+
+       @GuardedBy("lock")
+       private volatile boolean running;
+
+       /**
+        * Creates a ZooKeeperLeaderElectionDriver object.
+        *
+        * @param client Client which is connected to the ZooKeeper quorum
+        * @param latchPath ZooKeeper node path for the leader election latch
+        * @param leaderPath ZooKeeper node path for the node which stores the 
current leader information
+        * @param leaderElectionEventHandler Event handler for processing 
leader change events
+        * @param fatalErrorHandler Fatal error handler
+        * @param leaderContenderDescription Leader contender description
+        */
+       public ZooKeeperLeaderElectionDriver(
+                       CuratorFramework client,
+                       String latchPath,
+                       String leaderPath,
+                       LeaderElectionEventHandler leaderElectionEventHandler,
+                       FatalErrorHandler fatalErrorHandler,
+                       String leaderContenderDescription) throws Exception {
+               this.client = checkNotNull(client);
+               this.leaderPath = checkNotNull(leaderPath);
+               this.leaderElectionEventHandler = 
checkNotNull(leaderElectionEventHandler);
+               this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+               this.leaderContenderDescription = 
checkNotNull(leaderContenderDescription);
+
+               leaderLatch = new LeaderLatch(client, checkNotNull(latchPath));
+               cache = new NodeCache(client, leaderPath);
+
+               client.getUnhandledErrorListenable().addListener(this);
+
+               leaderLatch.addListener(this);
+               leaderLatch.start();
+
+               cache.getListenable().addListener(this);
+               cache.start();
+
+               client.getConnectionStateListenable().addListener(listener);
+
+               running = true;
+       }
+
+       @Override
+       public void close() throws Exception{
+               synchronized (lock) {
+                       if (!running) {
+                               return;
+                       }
+                       running = false;
+
+                       LOG.info("Closing {}", this);
+
+                       
client.getUnhandledErrorListenable().removeListener(this);
+
+                       
client.getConnectionStateListenable().removeListener(listener);
+
+                       Exception exception = null;
+
+                       try {
+                               cache.close();
+                       } catch (Exception e) {
+                               exception = e;
+                       }
+
+                       try {
+                               leaderLatch.close();
+                       } catch (Exception e) {
+                               exception = ExceptionUtils.firstOrSuppressed(e, 
exception);
+                       }
+
+                       if (exception != null) {
+                               throw new Exception("Could not properly stop 
the ZooKeeperLeaderElectionDriver.", exception);
+                       }
+               }
+       }
+
+       @Override
+       public boolean hasLeadership() {
+               return leaderLatch.hasLeadership();
+       }
+
+       @Override
+       public void isLeader() {
+               synchronized (lock) {
+                       if (running) {
+                               leaderElectionEventHandler.onGrantLeadership();
+                       }
+               }

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




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

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


Reply via email to