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 2010/05/01 00:26:03 UTC

svn commit: r939848 - in /hadoop/hbase/trunk: contrib/stargate/core/src/test/java/org/apache/hadoop/hbase/stargate/ contrib/transactional/src/test/java/org/apache/hadoop/hbase/regionserver/transactional/ core/src/main/java/org/apache/hadoop/hbase/util/

Author: stack
Date: Fri Apr 30 22:26:03 2010
New Revision: 939848

URL: http://svn.apache.org/viewvc?rev=939848&view=rev
Log:
Fixing broken build... forgot to add JVMClusterUtil, etc.

Added:
    hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
Modified:
    hadoop/hbase/trunk/contrib/stargate/core/src/test/java/org/apache/hadoop/hbase/stargate/MiniClusterTestBase.java
    hadoop/hbase/trunk/contrib/transactional/src/test/java/org/apache/hadoop/hbase/regionserver/transactional/TestTHLogRecovery.java

Modified: hadoop/hbase/trunk/contrib/stargate/core/src/test/java/org/apache/hadoop/hbase/stargate/MiniClusterTestBase.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/contrib/stargate/core/src/test/java/org/apache/hadoop/hbase/stargate/MiniClusterTestBase.java?rev=939848&r1=939847&r2=939848&view=diff
==============================================================================
--- hadoop/hbase/trunk/contrib/stargate/core/src/test/java/org/apache/hadoop/hbase/stargate/MiniClusterTestBase.java (original)
+++ hadoop/hbase/trunk/contrib/stargate/core/src/test/java/org/apache/hadoop/hbase/stargate/MiniClusterTestBase.java Fri Apr 30 22:26:03 2010
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.MiniZooKe
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;

Modified: hadoop/hbase/trunk/contrib/transactional/src/test/java/org/apache/hadoop/hbase/regionserver/transactional/TestTHLogRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/contrib/transactional/src/test/java/org/apache/hadoop/hbase/regionserver/transactional/TestTHLogRecovery.java?rev=939848&r1=939847&r2=939848&view=diff
==============================================================================
--- hadoop/hbase/trunk/contrib/transactional/src/test/java/org/apache/hadoop/hbase/regionserver/transactional/TestTHLogRecovery.java (original)
+++ hadoop/hbase/trunk/contrib/transactional/src/test/java/org/apache/hadoop/hbase/regionserver/transactional/TestTHLogRecovery.java Fri Apr 30 22:26:03 2010
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.ipc.Trans
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
 
 public class TestTHLogRecovery extends HBaseClusterTestCase {
   private static final Log LOG = LogFactory.getLog(TestTHLogRecovery.class);
@@ -141,8 +142,8 @@ public class TestTHLogRecovery extends H
   // }
 
   private void flushRegionServer() {
-    List<LocalHBaseCluster.RegionServerThread> regionThreads = cluster
-        .getRegionThreads();
+    List<JVMClusterUtil.RegionServerThread> regionThreads = cluster
+        .getRegionServerThreads();
 
     HRegion region = null;
     int server = -1;
@@ -171,8 +172,8 @@ public class TestTHLogRecovery extends H
    * just shut down.
    */
   private void stopOrAbortRegionServer(final boolean abort) {
-    List<LocalHBaseCluster.RegionServerThread> regionThreads = cluster
-        .getRegionThreads();
+    List<JVMClusterUtil.RegionServerThread> regionThreads = cluster
+        .getRegionServerThreads();
 
     int server = -1;
     for (int i = 0; i < regionThreads.size(); i++) {

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java?rev=939848&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java Fri Apr 30 22:26:03 2010
@@ -0,0 +1,162 @@
+/**
+ * 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.util;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+
+/**
+ * Utility used running a cluster all in the one JVM.
+ */
+public class JVMClusterUtil {
+  private static final Log LOG = LogFactory.getLog(JVMClusterUtil.class);
+
+  /**
+   * Datastructure to hold RegionServer Thread and RegionServer instance
+   */
+  public static class RegionServerThread extends Thread {
+    private final HRegionServer regionServer;
+    
+    public RegionServerThread(final HRegionServer r, final int index) {
+      super(r, "RegionServer:" + index);
+      this.regionServer = r;
+    }
+
+    /** @return the region server */
+    public HRegionServer getRegionServer() {
+      return this.regionServer;
+    }
+    
+    /**
+     * Block until the region server has come online, indicating it is ready
+     * to be used.
+     */
+    public void waitForServerOnline() {
+      while (!regionServer.isOnline()) {
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          // continue waiting
+        }
+      }
+    }
+  }
+
+  /**
+   * Creates a {@link RegionServerThread}.
+   * Call 'start' on the returned thread to make it run.
+   * @param c Configuration to use.
+   * @param hrsc Class to create.
+   * @param index Used distingushing the object returned.
+   * @throws IOException
+   * @return Region server added.
+   */
+  public static JVMClusterUtil.RegionServerThread createRegionServerThread(final HBaseConfiguration c,
+    final Class<? extends HRegionServer> hrsc, final int index)
+  throws IOException {
+      HRegionServer server; 
+      try {
+        server = hrsc.getConstructor(HBaseConfiguration.class).newInstance(c);
+      } catch (Exception e) {
+        IOException ioe = new IOException();
+        ioe.initCause(e);
+        throw ioe;
+      }
+      return new JVMClusterUtil.RegionServerThread(server, index);
+  }
+
+  /**
+   * Start the cluster.
+   * @param m
+   * @param regionServers
+   * @return Address to use contacting master.
+   */
+  public static String startup(final HMaster m,
+      final List<JVMClusterUtil.RegionServerThread> regionservers) {
+    if (m != null) m.start();
+    if (regionservers != null) {
+      for (JVMClusterUtil.RegionServerThread t: regionservers) {
+        t.start();
+      }
+    }
+    return m == null? null: m.getMasterAddress().toString();
+  }
+
+  /**
+   * @param master
+   * @param regionservers
+   */
+  public static void shutdown(final HMaster master,
+      final List<RegionServerThread> regionservers) {
+    LOG.debug("Shutting down HBase Cluster");
+    // Be careful how the hdfs shutdown thread runs in context where more than
+    // one regionserver in the mix.
+    Thread hdfsClientFinalizer = null;
+    for (JVMClusterUtil.RegionServerThread t: regionservers) {
+      Thread tt = t.getRegionServer().setHDFSShutdownThreadOnExit(null);
+      if (hdfsClientFinalizer == null && tt != null) {
+        hdfsClientFinalizer = tt;
+      }
+    }
+    if (master != null) {
+      master.shutdown();
+    }
+    // regionServerThreads can never be null because they are initialized when
+    // the class is constructed.
+      for(Thread t: regionservers) {
+        if (t.isAlive()) {
+          try {
+            t.join();
+          } catch (InterruptedException e) {
+            // continue
+          }
+        }
+      }
+    if (master != null) {
+      while (master.isAlive()) {
+        try {
+          // The below has been replaced to debug sometime hangs on end of
+          // tests.
+          // this.master.join():
+          Threads.threadDumpingIsAlive(master);
+        } catch(InterruptedException e) {
+          // continue
+        }
+      }
+    }
+    if (hdfsClientFinalizer != null) {
+      // Don't run the shutdown thread.  Plays havoc if we try to start a
+      // minihbasecluster immediately after this one has gone down (In
+      // Filesystem, the shutdown thread is kept in a static and is created
+      // on classloading.  Can only run it once).
+      // hdfsClientFinalizer.start();
+      // Threads.shutdown(hdfsClientFinalizer);
+    }
+    LOG.info("Shutdown " +
+      ((regionservers != null)? master.getName(): "0 masters") +
+      " " + regionservers.size() + " region server(s)");
+  }
+}
\ No newline at end of file