You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jg...@apache.org on 2010/07/06 23:56:28 UTC

svn commit: r961028 [2/2] - in /hbase/branches/0.90_master_rewrite: ./ src/main/java/org/apache/hadoop/hbase/ src/main/java/org/apache/hadoop/hbase/client/ src/main/java/org/apache/hadoop/hbase/master/ src/main/java/org/apache/hadoop/hbase/regionserver...

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java?rev=961028&r1=961027&r2=961028&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java Tue Jul  6 21:56:27 2010
@@ -85,15 +85,13 @@ import org.apache.zookeeper.data.Stat;
  *                         in transition. The first byte of the data for each 
  *                         of these nodes is the event type. This is used to 
  *                         deserialize the rest of the data.
+ *                         
+ * TODO: Eventually this class will be removed and everything put into ZKUtil
+ *       and ZooKeeperWatcher.  Can rename stuff once it's done.
  */
 public class ZooKeeperWrapper implements Watcher {
   protected static final Log LOG = LogFactory.getLog(ZooKeeperWrapper.class);
 
-  // instances of the watcher
-  private static Map<String,ZooKeeperWrapper> INSTANCES = 
-    new HashMap<String,ZooKeeperWrapper>();
-  // lock for ensuring a singleton per instance type
-  private static Lock createLock = new ReentrantLock();
   // name of this instance
   private String instanceName;
 
@@ -136,35 +134,6 @@ public class ZooKeeperWrapper implements
 
   private List<Watcher> listeners = Collections.synchronizedList(new ArrayList<Watcher>());
 
-  // return the singleton given the name of the instance
-  public static ZooKeeperWrapper getInstance(Configuration conf, String name) {
-    name = getZookeeperClusterKey(conf, name);
-    return INSTANCES.get(name);
-  }
-  // creates only one instance
-  public static ZooKeeperWrapper createInstance(Configuration conf, String name) {
-    if (getInstance(conf, name) != null) {
-      return getInstance(conf, name);
-    }
-    ZooKeeperWrapper.createLock.lock();
-    try {
-      if (getInstance(conf, name) == null) {
-        try {
-          String fullname = getZookeeperClusterKey(conf, name);
-          ZooKeeperWrapper instance = new ZooKeeperWrapper(conf, fullname);
-          INSTANCES.put(fullname, instance);
-        }
-        catch (Exception e) {
-          LOG.error("<" + name + ">" + "Error creating a ZooKeeperWrapper " + e);
-        }
-      }
-    }
-    finally {
-      createLock.unlock();
-    }
-    return getInstance(conf, name);
-  }
-
   /**
    * Create a ZooKeeperWrapper. The Zookeeper wrapper listens to all messages
    * from Zookeeper, and notifies all the listeners about all the messages. Any
@@ -172,13 +141,14 @@ public class ZooKeeperWrapper implements
    * and remove itself from being a listener.
    *
    * @param conf HBaseConfiguration to read settings from.
+   * @param instanceName name of this zk instance, used for debugging
    * @throws IOException If a connection error occurs.
    */
-  private ZooKeeperWrapper(Configuration conf, String instanceName)
+  protected ZooKeeperWrapper(Configuration conf, String instanceName)
   throws IOException {
     this.instanceName = instanceName;
-    Properties properties = HQuorumPeer.makeZKProps(conf);
-    quorumServers = HQuorumPeer.getZKQuorumServersString(properties);
+    Properties properties = ZKConfig.makeZKProps(conf);
+    quorumServers = ZKConfig.getZKQuorumServersString(properties);
     if (quorumServers == null) {
       throw new IOException("Could not read quorum servers from " +
                             HConstants.ZOOKEEPER_CONFIG_NAME);
@@ -192,7 +162,7 @@ public class ZooKeeperWrapper implements
     String rsZNodeName         = conf.get("zookeeper.znode.rs", "rs");
     String masterAddressZNodeName = conf.get("zookeeper.znode.master", "master");
     String stateZNodeName      = conf.get("zookeeper.znode.state", "shutdown");
-    String regionsInTransitZNodeName = conf.get("zookeeper.znode.regionInTransition", "UNASSIGNED");
+    String regionsInTransitZNodeName = conf.get("zookeeper.znode.regionInTransition", "unassigned");
 
     rootRegionZNode     = getZNode(parentZNode, rootServerZNodeName);
     rsZNode             = getZNode(parentZNode, rsZNodeName);
@@ -243,7 +213,6 @@ public class ZooKeeperWrapper implements
   }
 
   /** @return String dump of everything in ZooKeeper. */
-  @SuppressWarnings({"ConstantConditions"})
   public String dump() {
     StringBuilder sb = new StringBuilder();
     sb.append("\nHBase tree in ZooKeeper is rooted at ").append(parentZNode);
@@ -767,7 +736,6 @@ public class ZooKeeperWrapper implements
   public void close() {
     try {
       zooKeeper.close();
-      INSTANCES.remove(instanceName);
       LOG.debug("<" + instanceName + ">" + "Closed connection with ZooKeeper; " + this.rootRegionZNode);
     } catch (InterruptedException e) {
       LOG.warn("<" + instanceName + ">" + "Failed to close connection with ZooKeeper");

Modified: hbase/branches/0.90_master_rewrite/src/main/resources/hbase-webapps/regionserver/regionserver.jsp
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/resources/hbase-webapps/regionserver/regionserver.jsp?rev=961028&r1=961027&r2=961028&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/resources/hbase-webapps/regionserver/regionserver.jsp (original)
+++ hbase/branches/0.90_master_rewrite/src/main/resources/hbase-webapps/regionserver/regionserver.jsp Tue Jul  6 21:56:27 2010
@@ -42,7 +42,7 @@
 <tr><td>HBase Version</td><td><%= org.apache.hadoop.hbase.util.VersionInfo.getVersion() %>, r<%= org.apache.hadoop.hbase.util.VersionInfo.getRevision() %></td><td>HBase version and svn revision</td></tr>
 <tr><td>HBase Compiled</td><td><%= org.apache.hadoop.hbase.util.VersionInfo.getDate() %>, <%= org.apache.hadoop.hbase.util.VersionInfo.getUser() %></td><td>When HBase version was compiled and by whom</td></tr>
 <tr><td>Metrics</td><td><%= metrics.toString() %></td><td>RegionServer Metrics; file and heap sizes are in megabytes</td></tr>
-<tr><td>Zookeeper Quorum</td><td><%= regionServer.getZooKeeperWrapper().getQuorumServers() %></td><td>Addresses of all registered ZK servers</td></tr>
+<tr><td>Zookeeper Quorum</td><td><%= regionServer.getZooKeeper().getQuorumServers() %></td><td>Addresses of all registered ZK servers</td></tr>
 </table>
 
 <h2>Online Regions</h2>

Modified: hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=961028&r1=961027&r2=961028&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Tue Jul  6 21:56:27 2010
@@ -52,15 +52,17 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.mapred.MiniMRCluster;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.zookeeper.ZooKeeper;
 
 import com.google.common.base.Preconditions;
 
@@ -693,9 +695,7 @@ public class HBaseTestingUtility {
    */
   public void expireMasterSession() throws Exception {
     HMaster master = hbaseCluster.getMaster();
-    ZooKeeperWrapper zkw = 
-      ZooKeeperWrapper.getInstance(conf, master.getHServerAddress().toString());
-    expireSession(zkw);
+    expireSession(master.getZooKeeper(), master);
   }
 
   /**
@@ -705,15 +705,15 @@ public class HBaseTestingUtility {
    */
   public void expireRegionServerSession(int index) throws Exception {
     HRegionServer rs = hbaseCluster.getRegionServer(index);
-    expireSession(rs.getZooKeeperWrapper());
+    expireSession(rs.getZooKeeper(), rs);
   }
 
-  public void expireSession(ZooKeeperWrapper nodeZK) throws Exception{
-    ZooKeeperWrapper zkw =
-        ZooKeeperWrapper.createInstance(conf,
-            ZooKeeperWrapper.class.getName());
+  public void expireSession(ZooKeeperWatcher nodeZK, ServerStatus server)
+  throws Exception {
+    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, 
+        ZooKeeperWatcher.class.getName(), server);
     zkw.registerListener(EmptyWatcher.instance);
-    String quorumServers = zkw.getQuorumServers();
+    String quorumServers = ZKConfig.getZKQuorumServersString(conf);
     int sessionTimeout = 5 * 1000; // 5 seconds
 
     byte[] password = nodeZK.getSessionPassword();

Modified: hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java?rev=961028&r1=961027&r2=961028&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (original)
+++ hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java Tue Jul  6 21:56:27 2010
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.client.HC
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
@@ -91,8 +92,8 @@ public class TestZooKeeper {
       throws IOException, InterruptedException {
     new HTable(conf, HConstants.META_TABLE_NAME);
 
-    ZooKeeperWrapper zkw =
-        ZooKeeperWrapper.createInstance(conf, TestZooKeeper.class.getName());
+    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, 
+        TestZooKeeper.class.getName(), null);
     zkw.registerListener(EmptyWatcher.instance);
     String quorumServers = zkw.getQuorumServers();
     int sessionTimeout = 5 * 1000; // 5 seconds
@@ -110,15 +111,17 @@ public class TestZooKeeper {
     System.err.println("ZooKeeper should have timed out");
     connection.relocateRegion(HConstants.ROOT_TABLE_NAME, HConstants.EMPTY_BYTE_ARRAY);
   }
-  @Test
-  public void testRegionServerSessionExpired() throws Exception{
+  
+//  @Test Disabled, we don't expect these to restart anymore
+  public void disabledTestRegionServerSessionExpired() throws Exception{
     LOG.info("Starting testRegionServerSessionExpired");
     new HTable(conf, HConstants.META_TABLE_NAME);
     TEST_UTIL.expireRegionServerSession(0);
     testSanity();
   }
-  @Test
-  public void testMasterSessionExpired() throws Exception {
+
+//@Test Disabled, we don't expect these to restart anymore
+  public void disabledTestMasterSessionExpired() throws Exception {
     LOG.info("Starting testMasterSessionExpired");
     new HTable(conf, HConstants.META_TABLE_NAME);
     TEST_UTIL.expireMasterSession();
@@ -179,8 +182,8 @@ public class TestZooKeeper {
    */
   @Test
   public void testZNodeDeletes() throws Exception {
-    ZooKeeperWrapper zkw =
-        ZooKeeperWrapper.createInstance(conf, TestZooKeeper.class.getName());
+    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, 
+        TestZooKeeper.class.getName(), null);
     zkw.registerListener(EmptyWatcher.instance);
     zkw.ensureExists("/l1/l2/l3/l4");
     try {

Modified: hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java?rev=961028&r1=961027&r2=961028&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java (original)
+++ hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java Tue Jul  6 21:56:27 2010
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.executor.
 import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -41,7 +41,7 @@ public class TestRestartCluster {
   private static final Log LOG = LogFactory.getLog(TestRestartCluster.class);
   private static Configuration conf;
   private static HBaseTestingUtility utility;
-  private static ZooKeeperWrapper zkWrapper;
+  private static ZooKeeperWatcher zooKeeper;
   private static final byte[] TABLENAME = Bytes.toBytes("master_transitions");
   private static final byte [][] FAMILIES = new byte [][] {Bytes.toBytes("a")};
   
@@ -59,11 +59,11 @@ public class TestRestartCluster {
 
   @Test (timeout=300000) public void testRestartClusterAfterKill()throws Exception {
     utility.startMiniZKCluster();
-    zkWrapper = ZooKeeperWrapper.createInstance(conf, "cluster1");
+    zooKeeper = new ZooKeeperWatcher(conf, "cluster1", null);
 
     // create the unassigned region, throw up a region opened state for META
-    String unassignedZNode = zkWrapper.getRegionInTransitionZNode();
-    zkWrapper.createZNodeIfNotExists(unassignedZNode);
+    String unassignedZNode = zooKeeper.getRegionInTransitionZNode();
+    zooKeeper.createZNodeIfNotExists(unassignedZNode);
     byte[] data = null;
     HBaseEventType hbEventType = HBaseEventType.RS2ZK_REGION_OPENED;
     try {
@@ -71,8 +71,8 @@ public class TestRestartCluster {
     } catch (IOException e) {
       LOG.error("Error creating event data for " + hbEventType, e);
     }
-    zkWrapper.createUnassignedRegion(HRegionInfo.ROOT_REGIONINFO.getEncodedName(), data);
-    zkWrapper.createUnassignedRegion(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName(), data);
+    zooKeeper.createUnassignedRegion(HRegionInfo.ROOT_REGIONINFO.getEncodedName(), data);
+    zooKeeper.createUnassignedRegion(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName(), data);
     LOG.debug("Created UNASSIGNED zNode for ROOT and META regions in state " + HBaseEventType.M2ZK_REGION_OFFLINE);
     
     // start the HB cluster

Added: hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java?rev=961028&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java (added)
+++ hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java Tue Jul  6 21:56:27 2010
@@ -0,0 +1,117 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.regionserver;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.concurrent.Semaphore;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.CreateMode;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestMasterAddressManager {
+  private static final Log LOG = LogFactory.getLog(TestMasterAddressManager.class);
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniZKCluster();
+  }
+  
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniZKCluster();
+  }
+  /**
+   * Unit tests that uses ZooKeeper but does not use the master-side methods
+   * but rather acts directly on ZK.
+   * @throws Exception
+   */
+  @Test
+  public void testMasterAddressManagerFromZK() throws Exception {
+    
+    ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), 
+        "testMasterAddressManagerFromZK", null);
+    zk.createZNodeIfNotExists(zk.baseZNode);
+    
+    // Should not have a master yet
+    MasterAddressManager addressManager = new MasterAddressManager(zk);
+    addressManager.monitorMaster();
+    assertFalse(addressManager.hasMaster());
+    zk.registerListener(addressManager);
+    
+    // Use a listener to capture when the node is actually created
+    NodeCreationListener listener = new NodeCreationListener(zk, zk.masterAddressZNode);
+    zk.registerListener(listener);
+    
+    // Create the master node with a dummy address
+    String host = "hostname";
+    int port = 1234;
+    HServerAddress dummyAddress = new HServerAddress(host, port);
+    LOG.info("Creating master node");
+    zk.createZNodeIfNotExists(zk.masterAddressZNode, 
+        Bytes.toBytes(dummyAddress.toString()), CreateMode.EPHEMERAL, false);
+    
+    // Wait for the node to be created
+    LOG.info("Waiting for master address manager to be notified");
+    listener.waitForCreation();
+    LOG.info("Master node created");
+    assertTrue(addressManager.hasMaster());
+    HServerAddress pulledAddress = addressManager.getMasterAddress();
+    assertTrue(pulledAddress.equals(dummyAddress));
+    
+  }
+  
+  public static class NodeCreationListener extends ZooKeeperListener {
+    private static final Log LOG = LogFactory.getLog(NodeCreationListener.class);
+    
+    private Semaphore lock;
+    private String node;
+    
+    public NodeCreationListener(ZooKeeperWatcher watcher, String node) {
+      super(watcher);
+      lock = new Semaphore(0);
+      this.node = node;
+    }
+    
+    @Override
+    public void nodeCreated(String path) {
+      if(path.equals(node)) {
+        LOG.debug("nodeCreated(" + path + ")");
+        lock.release();
+      }
+    }
+    
+    public void waitForCreation() throws InterruptedException {
+      lock.acquire();
+    }
+  }
+}

Modified: hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java?rev=961028&r1=961027&r2=961028&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java (original)
+++ hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java Tue Jul  6 21:56:27 2010
@@ -64,7 +64,7 @@ public class TestHQuorumPeer extends HBa
 
   /** */
   public void testMakeZKProps() {
-    Properties properties = HQuorumPeer.makeZKProps(conf);
+    Properties properties = ZKConfig.makeZKProps(conf);
     assertEquals(dataDir.toString(), properties.get("dataDir"));
     assertEquals(Integer.valueOf(21810), Integer.valueOf(properties.getProperty("clientPort")));
     assertEquals("localhost:2888:3888", properties.get("server.0"));
@@ -72,7 +72,7 @@ public class TestHQuorumPeer extends HBa
 
     String oldValue = conf.get(HConstants.ZOOKEEPER_QUORUM);
     conf.set(HConstants.ZOOKEEPER_QUORUM, "a.foo.bar,b.foo.bar,c.foo.bar");
-    properties = HQuorumPeer.makeZKProps(conf);
+    properties = ZKConfig.makeZKProps(conf);
     assertEquals(dataDir.toString(), properties.get("dataDir"));
     assertEquals(Integer.valueOf(21810), Integer.valueOf(properties.getProperty("clientPort")));
     assertEquals("a.foo.bar:2888:3888", properties.get("server.0"));
@@ -91,7 +91,7 @@ public class TestHQuorumPeer extends HBa
 
     System.setProperty("hbase.master.hostname", "localhost");
     InputStream is = new ByteArrayInputStream(s.getBytes());
-    Properties properties = HQuorumPeer.parseZooCfg(conf, is);
+    Properties properties = ZKConfig.parseZooCfg(conf, is);
 
     assertEquals(dataDir.toString(), properties.get("dataDir"));
     assertEquals(Integer.valueOf(2181), Integer.valueOf(properties.getProperty("clientPort")));
@@ -111,7 +111,7 @@ public class TestHQuorumPeer extends HBa
     // Override with system property.
     System.setProperty("hbase.master.hostname", "foo.bar");
     is = new ByteArrayInputStream(s.getBytes());
-    properties = HQuorumPeer.parseZooCfg(conf, is);
+    properties = ZKConfig.parseZooCfg(conf, is);
     assertEquals("foo.bar:2888:3888", properties.get("server.0"));
 
     config.parseProperties(properties);
@@ -127,7 +127,7 @@ public class TestHQuorumPeer extends HBa
   public void testShouldAssignDefaultZookeeperClientPort() {
     Configuration config = HBaseConfiguration.create();
     config.clear();
-    Properties p = HQuorumPeer.makeZKProps(config);
+    Properties p = ZKConfig.makeZKProps(config);
     assertNotNull(p);
     assertEquals(2181, p.get("hbase.zookeeper.property.clientPort"));
   }