HBASE-3163 If we timeout PENDING_CLOSE and send another closeRegion RPC, need to handle NSRE from RS (comes as a RemoteException)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1029119 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Gray 2010-10-30 17:02:18 +00:00
parent 5abb3867c0
commit f14f3b216c
2 changed files with 24 additions and 16 deletions

View File

@ -629,6 +629,8 @@ Release 0.21.0 - Unreleased
HBASE-2406 Define semantics of cell timestamps/versions HBASE-2406 Define semantics of cell timestamps/versions
HBASE-3175 Commit of HBASE-3160 broke TestPriorityCompactionQueue up on HBASE-3175 Commit of HBASE-3160 broke TestPriorityCompactionQueue up on
hudson (nicolas via jgray) 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 IMPROVEMENTS

View File

@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.NodeAndData; import org.apache.hadoop.hbase.zookeeper.ZKUtil.NodeAndData;
import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.Writable;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.KeeperException.NoNodeException;
@ -987,11 +988,28 @@ public class AssignmentManager extends ZooKeeperListener {
} }
// Send CLOSE RPC // Send CLOSE RPC
try { 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())) { 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) { } catch (NotServingRegionException nsre) {
// 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 (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 // Did not CLOSE, so set region offline and assign it
LOG.debug("Attempted to send CLOSE to " + regions.get(region) + LOG.debug("Attempted to send CLOSE to " + regions.get(region) +
" for region " + region.getRegionNameAsString() + " but failed, " + " for region " + region.getRegionNameAsString() + " but failed, " +
@ -1000,18 +1018,6 @@ public class AssignmentManager extends ZooKeeperListener {
forceRegionStateToOffline(region); forceRegionStateToOffline(region);
assign(region); assign(region);
} }
} 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);
}
} }
/** /**