carp84 commented on a change in pull request #2614:
URL: https://github.com/apache/hbase/pull/2614#discussion_r516742100



##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/zksyncer/ClientZKSyncer.java
##########
@@ -145,8 +177,7 @@ private final void setDataForClientZkUntilSuccess(String 
node, byte[] data)
           LOG.warn(
             "Failed to create znode " + node + " due to: " + e.getMessage() + 
", will retry later");
         }
-      } catch (KeeperException.ConnectionLossException
-          | KeeperException.SessionExpiredException ee) {
+      } catch (KeeperException.SessionExpiredException see) {

Review comment:
       So we choose not to reconnect if there are some issue on network 
stability that cause zookeeper connection loss? And it seems the 
`reconnectAfterExpiration` is not following the same handling on line 173-174

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/zksyncer/ClientZKSyncer.java
##########
@@ -34,30 +33,69 @@
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * Tracks the target znode(s) on server ZK cluster and synchronize them to 
client ZK cluster if
  * changed
  * <p/>
- * The target znode(s) is given through {@link #getNodesToWatch()} method
+ * The target znode(s) is given through {@link #getPathsToWatch()} method
  */
 @InterfaceAudience.Private
 public abstract class ClientZKSyncer extends ZKListener {
   private static final Logger LOG = 
LoggerFactory.getLogger(ClientZKSyncer.class);
   private final Server server;
   private final ZKWatcher clientZkWatcher;
+
+  private static final class ZKData {
+
+    byte[] data;
+
+    boolean delete = false;
+
+    synchronized void set(byte[] data) {
+      this.data = data;
+      notifyAll();
+    }
+
+    synchronized byte[] get() throws InterruptedException {
+      while (!delete && data == null) {
+        wait();
+      }
+      byte[] d = data;
+      data = null;
+      return d;
+    }
+
+    synchronized void delete() {
+      this.delete = true;
+      notifyAll();
+    }
+
+    synchronized boolean isDeleted() {
+      return delete;
+    }
+  }
+
   // We use queues and daemon threads to synchronize the data to client ZK 
cluster

Review comment:
       The newly introduced `ZKData` structure is more efficient, shall we 
update the comments here and add some javadoc for the `ZKData` class?

##########
File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
##########
@@ -255,7 +260,20 @@ public void testAsyncTable() throws Exception {
       Get get = new Get(row);
       Result result = table.get(get).get();
       LOG.debug("Result: " + Bytes.toString(result.getValue(family, 
qualifier)));
-      Assert.assertArrayEquals(value, result.getValue(family, qualifier));
+      assertArrayEquals(value, result.getValue(family, qualifier));
+    }
+  }
+
+  @Test
+  public void testChangeMetaReplicaCount() throws Exception {

Review comment:
       Maybe adding more logics to verify the `ClientZKSyncer` is working well 
when meta replica changes, especially when reducing from multiple to single 
replica?




----------------------------------------------------------------
This is an automated message from the 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