You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2012/05/18 00:51:01 UTC

svn commit: r1339908 - in /hbase/branches/0.92: ./ src/main/java/org/apache/hadoop/hbase/ src/main/java/org/apache/hadoop/hbase/zookeeper/ src/test/java/org/apache/hadoop/hbase/

Author: apurtell
Date: Thu May 17 22:51:00 2012
New Revision: 1339908

URL: http://svn.apache.org/viewvc?rev=1339908&view=rev
Log:
HBASE-6011. Unable to start master in local mode

Added:
    hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java
Modified:
    hbase/branches/0.92/CHANGES.txt
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java

Modified: hbase/branches/0.92/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/CHANGES.txt?rev=1339908&r1=1339907&r2=1339908&view=diff
==============================================================================
--- hbase/branches/0.92/CHANGES.txt (original)
+++ hbase/branches/0.92/CHANGES.txt Thu May 17 22:51:00 2012
@@ -63,6 +63,7 @@ Release 0.92.2 - Unreleased
    HBASE-5942  HConnnectionManager.getRegionServerWithRetries doesn't call afterCall properly
    HBASE-5922  HalfStoreFileReader seekBefore causes StackOverflowError (Nate Putnam)
    HBASE-6018  hbck fails with a RejectedExecutionException when >50 regions present
+   HBASE-6011  Unable to start master in local mode
 
   IMPROVEMENTS
    HBASE-5592  Make it easier to get a table from shell (Ben West)

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java?rev=1339908&r1=1339907&r2=1339908&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java Thu May 17 22:51:00 2012
@@ -67,6 +67,7 @@ public class LocalHBaseCluster {
   /** 'local:' */
   public static final String LOCAL_COLON = LOCAL + ":";
   private final Configuration conf;
+  private final Class<? extends HMaster> masterClass;
   private final Class<? extends HRegionServer> regionServerClass;
 
   /**
@@ -139,6 +140,8 @@ public class LocalHBaseCluster {
     // clash over default ports.
     conf.set(HConstants.MASTER_PORT, "0");
     conf.set(HConstants.REGIONSERVER_PORT, "0");
+    this.masterClass = (Class<? extends HMaster>)
+      conf.getClass(HConstants.MASTER_IMPL, masterClass);
     // Start the HMasters.
     for (int i = 0; i < noMasters; i++) {
       addMaster(new Configuration(conf), i);
@@ -192,7 +195,7 @@ public class LocalHBaseCluster {
     // its HConnection instance rather than share (see HBASE_INSTANCES down in
     // the guts of HConnectionManager.
     JVMClusterUtil.MasterThread mt = JVMClusterUtil.createMasterThread(c,
-        (Class<? extends HMaster>) c.getClass(HConstants.MASTER_IMPL, HMaster.class), index);
+        this.masterClass, index);
     this.masterThreads.add(mt);
     return mt;
   }

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java?rev=1339908&r1=1339907&r2=1339908&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java Thu May 17 22:51:00 2012
@@ -355,4 +355,11 @@ public class MiniZooKeeperCluster {
     }
     return false;
   }
+
+  /**
+   * @return Client port the minicluster is listening on
+   */
+  public int getClientPort() {
+    return clientPort;
+  }
 }

Added: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java?rev=1339908&view=auto
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java (added)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java Thu May 17 22:51:00 2012
@@ -0,0 +1,126 @@
+/**
+ * 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;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.apache.zookeeper.KeeperException;
+
+import org.junit.Test;
+
+public class TestLocalHBaseCluster {
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  /**
+   * Check that we can start a local HBase cluster specifying a custom master
+   * and regionserver class and then cast back to those classes; also that
+   * the cluster will launch and terminate cleanly. See HBASE-6011.
+   */
+  @Test
+  public void testLocalHBaseCluster() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    MiniZooKeeperCluster zkCluster = TEST_UTIL.startMiniZKCluster();
+    conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, Integer.toString(zkCluster.getClientPort()));
+    LocalHBaseCluster cluster = new LocalHBaseCluster(conf, 1, 1, MyHMaster.class,
+      MyHRegionServer.class);
+    // Can we cast back to our master class?
+    try {
+      ((MyHMaster)cluster.getMaster(0)).setZKCluster(zkCluster);
+    } catch (ClassCastException e) {
+      fail("Could not cast master to our class");
+    }
+    // Can we cast back to our regionserver class?
+    try {
+      ((MyHRegionServer)cluster.getRegionServer(0)).echo(42);
+    } catch (ClassCastException e) {
+      fail("Could not cast regionserver to our class");
+    }
+    // Does the cluster start successfully?
+    try {
+      cluster.startup();
+      waitForClusterUp(conf);
+    } catch (IOException e) {
+      fail("LocalHBaseCluster did not start successfully");
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private void waitForClusterUp(Configuration conf) throws IOException {
+    HTable t = new HTable(conf, HConstants.META_TABLE_NAME);
+    ResultScanner s = t.getScanner(new Scan());
+    while (s.next() != null) {
+      continue;
+    }
+    s.close();
+    t.close();
+  }
+
+  /**
+   * A private master class similar to that used by HMasterCommandLine when
+   * running in local mode.
+   */
+  public static class MyHMaster extends HMaster {
+    private MiniZooKeeperCluster zkcluster = null;
+
+    public MyHMaster(Configuration conf) throws IOException, KeeperException,
+        InterruptedException {
+      super(conf);
+    }
+
+    @Override
+    public void run() {
+      super.run();
+      if (this.zkcluster != null) {
+        try {
+          this.zkcluster.shutdown();
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      }
+    }
+
+    void setZKCluster(final MiniZooKeeperCluster zkcluster) {
+      this.zkcluster = zkcluster;
+    }
+  }
+
+  /**
+   * A private regionserver class with a dummy method for testing casts
+   */
+  public static class MyHRegionServer extends HRegionServer {
+
+    public MyHRegionServer(Configuration conf) throws IOException,
+        InterruptedException {
+      super(conf);
+    }
+
+    public int echo(int val) {
+      return val;
+    }
+  }
+}