HBASE-3621 The timeout handler in AssignmentManager does an RPC while holding lock on RIT; a big no-no

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1084059 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-03-22 03:11:31 +00:00
parent 53a031f00c
commit 99f9ca542f
2 changed files with 23 additions and 10 deletions

View File

@ -44,6 +44,8 @@ Release 0.91.0 - Unreleased
HBASE-3641 LruBlockCache.CacheStats.getHitCount() is not using the
correct variable
HBASE-3532 HRegion#equals is broken (Ted Yu via Stack)
HBASE-3621 The timeout handler in AssignmentManager does an RPC while
holding lock on RIT; a big no-no (Ted Yu via Stack)
IMPROVEMENTS
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)

View File

@ -1730,11 +1730,14 @@ public class AssignmentManager extends ZooKeeperListener {
protected void chore() {
// If bulkAssign in progress, suspend checks
if (this.bulkAssign) return;
List<HRegionInfo> unassigns = new ArrayList<HRegionInfo>();
Map<HRegionInfo, Boolean> assigns =
new HashMap<HRegionInfo, Boolean>();
synchronized (regionsInTransition) {
// Iterate all regions in transition checking for time outs
long now = System.currentTimeMillis();
for (RegionState regionState : regionsInTransition.values()) {
if(regionState.getStamp() + timeout <= now) {
if (regionState.getStamp() + timeout <= now) {
HRegionInfo regionInfo = regionState.getRegion();
LOG.info("Regions in transition timed out: " + regionState);
// Expired! Do a retry.
@ -1752,13 +1755,13 @@ public class AssignmentManager extends ZooKeeperListener {
LOG.info("Region has been OFFLINE for too long, " +
"reassigning " + regionInfo.getRegionNameAsString() +
" to a random server");
assign(regionState.getRegion(), false);
assigns.put(regionState.getRegion(), Boolean.FALSE);
break;
case PENDING_OPEN:
LOG.info("Region has been PENDING_OPEN for too " +
"long, reassigning region=" +
regionInfo.getRegionNameAsString());
assign(regionState.getRegion(), false, true);
assigns.put(regionState.getRegion(), Boolean.TRUE);
break;
case OPENING:
LOG.info("Region has been OPENING for too " +
@ -1794,7 +1797,7 @@ public class AssignmentManager extends ZooKeeperListener {
LOG.info("Successfully transitioned region=" +
regionInfo.getRegionNameAsString() + " into OFFLINE" +
" and forcing a new assignment");
assign(regionState, false, true);
assigns.put(regionState.getRegion(), Boolean.TRUE);
}
} catch (KeeperException.NoNodeException nne) {
// Node did not exist, can't time this out
@ -1817,16 +1820,17 @@ public class AssignmentManager extends ZooKeeperListener {
// If the server got the RPC, it will transition the node
// to CLOSING, so only do something here if no node exists
if (!ZKUtil.watchAndCheckExists(watcher,
ZKAssign.getNodeName(watcher,
regionInfo.getEncodedName()))) {
unassign(regionInfo, true);
ZKAssign.getNodeName(watcher, regionInfo.getEncodedName()))) {
// Queue running of an unassign -- do actual unassign
// outside of the regionsInTransition lock.
unassigns.add(regionInfo);
}
} catch (NoNodeException e) {
LOG.debug("Node no longer existed so not forcing another " +
"unassignment");
"unassignment");
} catch (KeeperException e) {
LOG.warn("Unexpected ZK exception timing out a region " +
"close", e);
"close", e);
}
break;
case CLOSING:
@ -1838,6 +1842,13 @@ public class AssignmentManager extends ZooKeeperListener {
}
}
}
// Finish the work for regions in PENDING_CLOSE state
for (HRegionInfo hri: unassigns) {
unassign(hri, true);
}
for (Map.Entry<HRegionInfo, Boolean> e: assigns.entrySet()){
assign(e.getKey(), false, e.getValue());
}
}
}
@ -2113,4 +2124,4 @@ public class AssignmentManager extends ZooKeeperListener {
public void stop() {
this.timeoutMonitor.interrupt();
}
}
}