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/08/19 05:37:41 UTC

svn commit: r1159494 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/master/HMaster.java src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java

Author: stack
Date: Fri Aug 19 03:37:41 2011
New Revision: 1159494

URL: http://svn.apache.org/viewvc?rev=1159494&view=rev
Log:
HBASE-4008 Problem while stopping HBase

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1159494&r1=1159493&r2=1159494&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Fri Aug 19 03:37:41 2011
@@ -203,6 +203,7 @@ Release 0.91.0 - Unreleased
    HBASE-4211  Do init-sizing of the StringBuilder making a ServerName
                (Benoît Sigoure)
    HBASE-4175  Fix FSUtils.createTableDescriptor() (Ramkrishna)
+   HBASE-4008  Problem while stopping HBase (Akash Ashok)
 
   IMPROVEMENTS
    HBASE-3290  Max Compaction Size (Nicolas Spiegelberg via Stack)  

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1159494&r1=1159493&r2=1159494&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Fri Aug 19 03:37:41 2011
@@ -1336,6 +1336,11 @@ implements HMasterInterface, HMasterRegi
     return this.stopped;
   }
 
+  boolean isAborted() {
+    return this.abort;
+  }
+  
+  
   /**
    * Report whether this master is currently the active master or not.
    * If not active master, we are parked on ZK waiting to become active.

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java?rev=1159494&r1=1159493&r2=1159494&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java Fri Aug 19 03:37:41 2011
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.MasterNot
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.ServerCommandLine;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.zookeeper.KeeperException;
@@ -138,6 +139,7 @@ public class HMasterCommandLine extends 
                                                           LocalHMaster.class, HRegionServer.class);
         ((LocalHMaster)cluster.getMaster(0)).setZKCluster(zooKeeperCluster);
         cluster.startup();
+        waitOnMasterThreads(cluster);
       } else {
         HMaster master = HMaster.constructMaster(masterClass, conf);
         if (master.isStopped()) {
@@ -146,6 +148,8 @@ public class HMasterCommandLine extends 
         }
         master.start();
         master.join();
+        if(master.isAborted())
+          throw new RuntimeException("HMaster Aborted");
       }
     } catch (Throwable t) {
       LOG.error("Failed to start master", t);
@@ -177,6 +181,27 @@ public class HMasterCommandLine extends 
     return 0;
   }
 
+  private void waitOnMasterThreads(LocalHBaseCluster cluster) throws InterruptedException{
+    List<JVMClusterUtil.MasterThread> masters = cluster.getMasters();
+    List<JVMClusterUtil.RegionServerThread> regionservers = cluster.getRegionServers();
+	  
+    if (masters != null) { 
+      for (JVMClusterUtil.MasterThread t : masters) {
+        t.join();
+        if(t.getMaster().isAborted()) {
+          closeAllRegionServerThreads(regionservers);
+          throw new RuntimeException("HMaster Aborted");
+        }
+      }
+    }
+  }
+
+  private static void closeAllRegionServerThreads(List<JVMClusterUtil.RegionServerThread> regionservers) {
+    for(JVMClusterUtil.RegionServerThread t : regionservers){
+      t.getRegionServer().stop("HMaster Aborted; Bringing down regions servers");
+    }
+  }
+  
   /*
    * Version of master that will shutdown the passed zk cluster on its way out.
    */
@@ -204,4 +229,4 @@ public class HMasterCommandLine extends 
       this.zkcluster = zkcluster;
     }
   }
-}
\ No newline at end of file
+}