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/06 00:11:39 UTC

svn commit: r941514 - in /hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master: HMaster.java RegionServerOperationQueue.java

Author: stack
Date: Wed May  5 22:11:38 2010
New Revision: 941514

URL: http://svn.apache.org/viewvc?rev=941514&view=rev
Log:
HBASE-2513 hbase-2414 added bug where we'd tight-loop if no root available

Modified:
    hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master/HMaster.java
    hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master/HMaster.java?rev=941514&r1=941513&r2=941514&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master/HMaster.java Wed May  5 22:11:38 2010
@@ -428,14 +428,17 @@ public class HMaster extends Thread impl
             break;
           }
         }
-        boolean doDelayQueue = this.regionManager.getRootRegionLocation() != null;
-        switch (this.regionServerOperationQueue.process(doDelayQueue)) {
+        final HServerAddress root = this.regionManager.getRootRegionLocation();
+        switch (this.regionServerOperationQueue.process(root)) {
         case FAILED:
+            // If FAILED op processing, bad. Exit.
           break FINISHED;
         case REQUEUED_BUT_PROBLEM:
           if (!checkFileSystem())
+              // If bad filesystem, exit.
             break FINISHED;
-        default: // PROCESSED, NOOP, REQUEUED:
+          default:
+            // Continue run loop if conditions are PROCESSED, NOOP, REQUEUED
           break;
         }
       }

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java?rev=941514&r1=941513&r2=941514&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java Wed May  5 22:11:38 2010
@@ -14,6 +14,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HMsg;
 import org.apache.hadoop.hbase.HServerInfo;
+import org.apache.hadoop.hbase.HServerAddress;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.util.Sleeper;
 import org.apache.hadoop.ipc.RemoteException;
@@ -90,19 +91,20 @@ public class RegionServerOperationQueue 
 
   /**
    * Try to get an operation off of the queue and process it.
-   * @param skipDelayedToDos If true, do not do delayed todos first but instead
-   * move straight to the current todos list.  This is set when we want to be
-   * sure that recently queued events are processed first such as the onlining
-   * of root region (Root region needs to be online before we can do meta
-   * onlining; meta onlining needs to be done before we can do... and so on). 
+   * @param rootRegionLocation Location of the root region.
    * @return {@link ProcessingResultCode#PROCESSED},
    * {@link ProcessingResultCode#REQUEUED},
    * {@link ProcessingResultCode#REQUEUED_BUT_PROBLEM}
    */ 
-  public synchronized ProcessingResultCode process(final boolean skipDelayedToDos) {
-    RegionServerOperation op = delayedToDoQueue.poll();
+  public synchronized ProcessingResultCode process(final HServerAddress rootRegionLocation) {
+    RegionServerOperation op = null;
+    // Only process the delayed queue if root region is online.  If offline,
+    // the operation to put it online is probably in the toDoQueue.  Process
+    // it first.
+    if (rootRegionLocation != null) {
+      op = delayedToDoQueue.poll();
+    } else {
     // if there aren't any todo items in the queue, sleep for a bit.
-    if (op == null) {
       try {
         op = toDoQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
       } catch (InterruptedException e) {