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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalDriver.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The counterpart to the {@link ZooKeeperLeaderElectionDriver}.
+ * {@link LeaderRetrievalService} implementation for Zookeeper. It retrieves 
the current leader which has
+ * been elected by the {@link ZooKeeperLeaderElectionDriver}.
+ * The leader address as well as the current leader session ID is retrieved 
from ZooKeeper.
+ */
+public class ZooKeeperLeaderRetrievalDriver implements LeaderRetrievalDriver, 
NodeCacheListener, UnhandledErrorListener {
+       private static final Logger LOG = 
LoggerFactory.getLogger(ZooKeeperLeaderRetrievalDriver.class);
+
+       /** Connection to the used ZooKeeper quorum. */
+       private final CuratorFramework client;
+
+       /** Curator recipe to watch changes of a specific ZooKeeper node. */
+       private final NodeCache cache;
+
+       private final String retrievalPath;
+
+       private final ConnectionStateListener connectionStateListener = 
(client, newState) -> handleStateChange(newState);
+
+       private final LeaderRetrievalEventHandler leaderRetrievalEventHandler;
+
+       private volatile boolean running;
+
+       /**
+        * Creates a leader retrieval service which uses ZooKeeper to retrieve 
the leader information.
+        *
+        * @param client Client which constitutes the connection to the 
ZooKeeper quorum
+        * @param retrievalPath Path of the ZooKeeper node which contains the 
leader information
+        * @param leaderRetrievalEventHandler handler to notify the leader 
changes.
+        */
+       public ZooKeeperLeaderRetrievalDriver(
+                       CuratorFramework client,
+                       String retrievalPath,
+                       LeaderRetrievalEventHandler 
leaderRetrievalEventHandler) throws Exception {
+               this.client = checkNotNull(client, "CuratorFramework client");
+               this.cache = new NodeCache(client, retrievalPath);
+               this.retrievalPath = checkNotNull(retrievalPath);
+
+               this.leaderRetrievalEventHandler = 
checkNotNull(leaderRetrievalEventHandler);
+
+               client.getUnhandledErrorListenable().addListener(this);
+               cache.getListenable().addListener(this);
+               cache.start();
+
+               
client.getConnectionStateListenable().addListener(connectionStateListener);
+
+               running = true;
+       }
+
+       @Override
+       public void close() throws Exception {
+               if (!running) {
+                       return;
+               }
+
+               running = false;

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




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

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


Reply via email to