HBASE-9481. Servershutdown handler get aborted with ConcurrentModificationException

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1521571 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Devaraj Das 2013-09-10 18:22:32 +00:00
parent 4745caaa87
commit 661d22ce59
1 changed files with 7 additions and 1 deletions

View File

@ -375,6 +375,7 @@ public class RegionStates {
regionAssignments.remove(region);
}
Set<HRegionInfo> regionsToOffline = new HashSet<HRegionInfo>();
for (RegionState state : regionsInTransition.values()) {
HRegionInfo hri = state.getRegion();
if (assignedRegions.contains(hri)) {
@ -387,7 +388,8 @@ public class RegionStates {
try {
// Delete the ZNode if exists
ZKAssign.deleteNodeFailSilent(watcher, hri);
regionOffline(hri);
// Offline regions outside the loop to avoid ConcurrentModificationException
regionsToOffline.add(hri);
} catch (KeeperException ke) {
server.abort("Unexpected ZK exception deleting node " + hri, ke);
}
@ -406,6 +408,10 @@ public class RegionStates {
}
}
}
for (HRegionInfo hri : regionsToOffline) {
regionOffline(hri);
}
assignedRegions.clear();
this.notifyAll();
return rits;