You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/03/22 00:34:54 UTC

svn commit: r1084021 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java

Author: stack
Date: Mon Mar 21 23:34:54 2011
New Revision: 1084021

URL: http://svn.apache.org/viewvc?rev=1084021&view=rev
Log:
HBASE-3052 Add ability to have multiple ZK servers in a quorum in MiniZooKeeperCluster for test writing -- BACKED IT OUT... NOT DONE YET

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1084021&r1=1084020&r2=1084021&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Mon Mar 21 23:34:54 2011
@@ -86,8 +86,6 @@ Release 0.91.0 - Unreleased
                needs an upper bound added (Ted Yu via Stack)
    HBASE-3676  Update region server load for AssignmentManager through
                regionServerReport() (Ted Yu via Stack)
-   HBASE-3052  Add ability to have multiple ZK servers in a quorum in
-               MiniZooKeeperCluster for test writing (Liyin Tang via Stack)
 
   TASK
    HBASE-3559  Move report of split to master OFF the heartbeat channel

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java?rev=1084021&r1=1084020&r2=1084021&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java Mon Mar 21 23:34:54 2011
@@ -28,8 +28,6 @@ import java.io.Reader;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
-import java.util.ArrayList;
-import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -50,21 +48,14 @@ public class MiniZooKeeperCluster {
   private static final int CONNECTION_TIMEOUT = 30000;
 
   private boolean started;
+  private int clientPort = 21818; // use non-standard port
 
-  private int defaultClientPort = 21818; // use non-standard port
-  private int clientPort = defaultClientPort; 
-  private int zooKeeperCandidateNum = 0;
-  
   private NIOServerCnxn.Factory standaloneServerFactory;
-  private int currentZooKeeper;
-  private List<ZooKeeperServer> zooKeeperServers;
   private int tickTime = 0;
 
   /** Create mini ZooKeeper cluster. */
   public MiniZooKeeperCluster() {
     this.started = false;
-    currentZooKeeper = -1;
-    zooKeeperServers = new ArrayList();
   }
 
   public void setClientPort(int clientPort) {
@@ -78,10 +69,6 @@ public class MiniZooKeeperCluster {
   public void setTickTime(int tickTime) {
     this.tickTime = tickTime;
   }
-  
-  public int getZooKeeperCandidateNum() {
-    return zooKeeperCandidateNum;
-  }
 
   // / XXX: From o.a.zk.t.ClientBase
   private static void setupTestEnv() {
@@ -92,41 +79,30 @@ public class MiniZooKeeperCluster {
     System.setProperty("zookeeper.preAllocSize", "100");
     FileTxnLog.setPreallocSize(100);
   }
-  
-  public int startup(File baseDir) throws IOException,
-  InterruptedException {
-    return startup(baseDir,1);
-  }
 
   /**
    * @param baseDir
-   * @param numZooKeeperServers
    * @return ClientPort server bound to.
    * @throws IOException
    * @throws InterruptedException
    */
-  public int startup(File baseDir, int numZooKeeperServers) throws IOException,
+  public int startup(File baseDir) throws IOException,
       InterruptedException {
-    if (numZooKeeperServers <= 0)
-      return -1;
 
     setupTestEnv();
 
     shutdown();
-    
-    for (int i = 0; i < numZooKeeperServers; i++) {
-      File dir = new File(baseDir, "zookeeper_"+i).getAbsoluteFile();
-      recreateDir(dir);
-  
-      int tickTimeToUse;
-      if (this.tickTime > 0) {
-        tickTimeToUse = this.tickTime;
-      } else {
-        tickTimeToUse = TICK_TIME;
-      }
-      ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
-      zooKeeperServers.add(server);
+
+    File dir = new File(baseDir, "zookeeper").getAbsoluteFile();
+    recreateDir(dir);
+
+    int tickTimeToUse;
+    if (this.tickTime > 0) {
+      tickTimeToUse = this.tickTime;
+    } else {
+      tickTimeToUse = TICK_TIME;
     }
+    ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
     while (true) {
       try {
         standaloneServerFactory =
@@ -139,14 +115,13 @@ public class MiniZooKeeperCluster {
       }
       break;
     }
-    currentZooKeeper = 0;
-    standaloneServerFactory.startup(zooKeeperServers.get(currentZooKeeper));   
+    standaloneServerFactory.startup(server);
+
     if (!waitForServerUp(clientPort, CONNECTION_TIMEOUT)) {
       throw new IOException("Waiting for startup of standalone server");
     }
 
     started = true;
-    zooKeeperCandidateNum = numZooKeeperServers-1;
     LOG.info("Started MiniZK Server on client port: " + clientPort);
     return clientPort;
   }
@@ -176,56 +151,6 @@ public class MiniZooKeeperCluster {
     }
 
     started = false;
-    zooKeeperCandidateNum = 0;
-    LOG.info("Shutdown MiniZK Server on client port: " + clientPort);
-  }
-  
-  /**@return clientPort return clientPort if there is another ZooKeeper Candidate can run; return
-   *         -1, if there is no candidates.
-   * @throws IOException
-   * @throws InterruptedException 
-   */
-  public int killCurrentZooKeeper() throws IOException, 
-                                        InterruptedException {
-    if (!started) {
-      return -1;
-    }
-    // Shutdown the current one
-    standaloneServerFactory.shutdown();
-    if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
-      throw new IOException("Waiting for shutdown of standalone server");
-    }
-    LOG.info("Kill the current MiniZK Server on client port: " 
-        + clientPort);
-    
-    if (zooKeeperCandidateNum == 0) {
-      return -1;
-    }
-    // Start another ZooKeeper Server
-    clientPort = defaultClientPort;
-    while (true) {
-      try {
-        standaloneServerFactory =
-          new NIOServerCnxn.Factory(new InetSocketAddress(clientPort));
-      } catch (BindException e) {
-        LOG.info("Failed binding ZK Server to client port: " + clientPort);
-        //this port is already in use. try to use another
-        clientPort++;
-        continue;
-      }
-      break;
-    }
-    currentZooKeeper = (++currentZooKeeper) % zooKeeperServers.size() ;
-    standaloneServerFactory.startup(zooKeeperServers.get(currentZooKeeper));
-
-    if (!waitForServerUp(clientPort, CONNECTION_TIMEOUT)) {
-      throw new IOException("Waiting for startup of standalone server");
-    }
-    
-    started = true;
-    zooKeeperCandidateNum--;
-    LOG.info("Started another candidate MiniZK Server on client port: " + clientPort);
-    return clientPort;
   }
 
   // XXX: From o.a.zk.t.ClientBase

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1084021&r1=1084020&r2=1084021&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Mon Mar 21 23:34:54 2011
@@ -245,38 +245,18 @@ public class HBaseTestingUtility {
    * @return zk cluster started.
    */
   public MiniZooKeeperCluster startMiniZKCluster() throws Exception {
-    return startMiniZKCluster(setupClusterTestBuildDir(),1);
+    return startMiniZKCluster(setupClusterTestBuildDir());
 
   }
-  
-  /**
-   * Call this if you only want a zk cluster.
-   * @param zooKeeperServerNum
-   * @see #startMiniZKCluster() if you want zk + dfs + hbase mini cluster.
-   * @throws Exception
-   * @see #shutdownMiniZKCluster()
-   * @return zk cluster started.
-   */
-  public MiniZooKeeperCluster startMiniZKCluster(int zooKeeperServerNum) 
-      throws Exception {
-    return startMiniZKCluster(setupClusterTestBuildDir(), zooKeeperServerNum);
 
-  }
-  
   private MiniZooKeeperCluster startMiniZKCluster(final File dir)
-    throws Exception {
-    return startMiniZKCluster(dir,1);
-  }
-  
-  private MiniZooKeeperCluster startMiniZKCluster(final File dir, 
-      int zooKeeperServerNum)
   throws Exception {
     this.passedZkCluster = false;
     if (this.zkCluster != null) {
       throw new IOException("Cluster already running at " + dir);
     }
     this.zkCluster = new MiniZooKeeperCluster();
-    int clientPort = this.zkCluster.startup(dir,zooKeeperServerNum);
+    int clientPort = this.zkCluster.startup(dir);
     this.conf.set("hbase.zookeeper.property.clientPort",
       Integer.toString(clientPort));
     return this.zkCluster;

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java?rev=1084021&r1=1084020&r2=1084021&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java Mon Mar 21 23:34:54 2011
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.client.HT
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -137,28 +136,6 @@ public class TestHBaseTestingUtility {
       cluster.shutdown();
     }
   }
-  
-  @Test public void testMiniZooKeeper() throws Exception {
-    MiniZooKeeperCluster cluster1 = this.hbt.startMiniZKCluster();
-    try {
-      assertEquals(0, cluster1.getZooKeeperCandidateNum());
-      assertTrue((cluster1.killCurrentZooKeeper() == -1));
-    } finally {
-      cluster1.shutdown();
-    }
-    
-    this.hbt.shutdownMiniZKCluster();
-    
-    MiniZooKeeperCluster cluster2 = this.hbt.startMiniZKCluster(5);
-    try {
-      assertEquals(4, cluster2.getZooKeeperCandidateNum());
-      assertTrue((cluster2.killCurrentZooKeeper() > 0));
-      assertTrue((cluster2.killCurrentZooKeeper() > 0));
-      assertEquals(2, cluster2.getZooKeeperCandidateNum());
-    } finally {
-      cluster2.shutdown();
-    }
-  }
 
   @Test public void testMiniDFSCluster() throws Exception {
     MiniDFSCluster cluster = this.hbt.startMiniDFSCluster(1);