HBASE-5120 Timeout monitor races with table disable handler

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1245731 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-02-17 20:03:57 +00:00
parent 6fa8f6a042
commit aa7a5b2926
1 changed files with 48 additions and 1 deletions

View File

@ -60,9 +60,11 @@ import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.catalog.RootLocationEditor;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState.State;
import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
@ -84,6 +86,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.data.Stat;
@ -1744,6 +1747,7 @@ public class AssignmentManager extends ZooKeeperListener {
// TODO: Method needs refactoring. Ugly buried returns throughout. Beware!
LOG.debug("Starting unassignment of region " +
region.getRegionNameAsString() + " (offlining)");
synchronized (this.regions) {
// Check if this region is currently assigned
if (!regions.containsKey(region)) {
@ -1817,6 +1821,24 @@ public class AssignmentManager extends ZooKeeperListener {
synchronized (this.regions) {
server = regions.get(region);
}
// ClosedRegionhandler can remove the server from this.regions
if (server == null) {
// Possibility of disable flow removing from RIT.
synchronized (regionsInTransition) {
state = regionsInTransition.get(encodedName);
if (state != null) {
// remove only if the state is PENDING_CLOSE or CLOSING
State presentState = state.getState();
if (presentState == State.PENDING_CLOSE
|| presentState == State.CLOSING) {
this.regionsInTransition.remove(encodedName);
}
}
}
// delete the node. if no node exists need not bother.
deleteClosingOrClosedNode(region);
return;
}
try {
// TODO: We should consider making this look more like it does for the
// region open where we catch all throwables and never abort
@ -1852,6 +1874,7 @@ public class AssignmentManager extends ZooKeeperListener {
synchronized (this.regions) {
this.regions.remove(region);
}
deleteClosingOrClosedNode(region);
}
}
// RS is already processing this region, only need to update the timestamp
@ -1866,6 +1889,30 @@ public class AssignmentManager extends ZooKeeperListener {
}
}
private void deleteClosingOrClosedNode(HRegionInfo region) {
try {
if (!ZKAssign.deleteNode(master.getZooKeeper(), region.getEncodedName(),
EventHandler.EventType.M_ZK_REGION_CLOSING)) {
boolean deleteNode = ZKAssign.deleteNode(master.getZooKeeper(), region
.getEncodedName(), EventHandler.EventType.RS_ZK_REGION_CLOSED);
// TODO : We don't abort if the delete node returns false. Is there any
// such corner case?
if (!deleteNode) {
LOG.error("The deletion of the CLOSED node for the region "
+ region.getEncodedName() + " returned " + deleteNode);
}
}
} catch (NoNodeException e) {
LOG.debug("CLOSING/CLOSED node for the region " + region.getEncodedName()
+ " already deleted");
} catch (KeeperException ke) {
master.abort(
"Unexpected ZK exception deleting node CLOSING/CLOSED for the region "
+ region.getEncodedName(), ke);
return;
}
}
/**
* @param path
* @return True if znode is in SPLIT or SPLITTING state.