hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jg...@apache.org
Subject svn commit: r1029119 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
Date Sat, 30 Oct 2010 17:02:19 GMT
Author: jgray
Date: Sat Oct 30 17:02:18 2010
New Revision: 1029119

URL: http://svn.apache.org/viewvc?rev=1029119&view=rev
Log:
HBASE-3163 If we timeout PENDING_CLOSE and send another closeRegion RPC, need to handle NSRE
from RS (comes as a RemoteException)

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

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1029119&r1=1029118&r2=1029119&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Sat Oct 30 17:02:18 2010
@@ -629,6 +629,8 @@ Release 0.21.0 - Unreleased
    HBASE-2406  Define semantics of cell timestamps/versions
    HBASE-3175  Commit of HBASE-3160 broke TestPriorityCompactionQueue up on
                hudson (nicolas via jgray)
+   HBASE-3163  If we timeout PENDING_CLOSE and send another closeRegion RPC,
+               need to handle NSRE from RS (comes as a RemoteException)
 
 
   IMPROVEMENTS

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java?rev=1029119&r1=1029118&r2=1029119&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java Sat Oct
30 17:02:18 2010
@@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.zookeeper
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.NodeAndData;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
@@ -987,31 +988,36 @@ public class AssignmentManager extends Z
     }
     // Send CLOSE RPC
     try {
-      if(!serverManager.sendRegionClose(regions.get(region),
+      // TODO: We should consider making this look more like it does for the
+      //       region open where we catch all throwables and never abort
+      if(serverManager.sendRegionClose(regions.get(region),
           state.getRegion())) {
-        throw new NotServingRegionException("Server failed to close region");
+        LOG.debug("Sent CLOSE to " + regions.get(region) + " for region " +
+            region.getRegionNameAsString());
+        return;
       }
     } catch (NotServingRegionException nsre) {
-      // Did not CLOSE, so set region offline and assign it
-      LOG.debug("Attempted to send CLOSE to " + regions.get(region) +
-          " for region " + region.getRegionNameAsString() + " but failed, " +
-          "setting region as OFFLINE and reassigning");
-      synchronized (regionsInTransition) {
-        forceRegionStateToOffline(region);
-        assign(region);
+      // Failed to close, so pass through and reassign
+    } catch (RemoteException re) {
+      if (re.unwrapRemoteException() instanceof NotServingRegionException) {
+        // Failed to close, so pass through and reassign
+      } else {
+        this.master.abort("Remote unexpected exception",
+            re.unwrapRemoteException());
       }
-    } catch (IOException e) {
-      // For now call abort if unexpected exception -- radical, but will get fellas attention.
-      // St.Ack 20101012
-      // I don't think IOE can happen anymore, only NSRE IOE is used here
-      // should be able to remove this at least.  jgray 20101024
-      // I lied, we actually get RemoteException wrapping our NSRE, need to unwrap
-      this.master.abort("Remote unexpected exception", e);
     } catch (Throwable t) {
       // For now call abort if unexpected exception -- radical, but will get fellas attention.
       // St.Ack 20101012
       this.master.abort("Remote unexpected exception", t);
     }
+    // Did not CLOSE, so set region offline and assign it
+    LOG.debug("Attempted to send CLOSE to " + regions.get(region) +
+        " for region " + region.getRegionNameAsString() + " but failed, " +
+        "setting region as OFFLINE and reassigning");
+    synchronized (regionsInTransition) {
+      forceRegionStateToOffline(region);
+      assign(region);
+    }
   }
 
   /**



Mime
View raw message