Index: src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java (revision 1201618) +++ src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java (working copy) @@ -103,7 +103,7 @@ public enum EventType { // Messages originating from RS (NOTE: there is NO direct communication from // RS to Master). These are a result of RS updates into ZK. - RS_ZK_REGION_CLOSING (1), // RS is in process of closing a region + RS_ZK_REGION_CLOSING (1), // Master adds this region as closing in ZK(HBASE-3789) RS_ZK_REGION_CLOSED (2), // RS has finished closing a region RS_ZK_REGION_OPENING (3), // RS is in process of opening a region RS_ZK_REGION_OPENED (4), // RS has finished opening a region Index: src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (revision 1201618) +++ src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (working copy) @@ -456,19 +456,19 @@ synchronized (regionsInTransition) { switch (data.getEventType()) { case RS_ZK_REGION_CLOSING: - // If zk node of the region was updated by a live server skip this - // region and just add it into RIT. - if (isOnDeadServer(regionInfo, deadServers) && - (data.getOrigin() == null || !serverManager.isServerOnline(data.getOrigin()))) { + + if (isOnDeadServer(regionInfo, deadServers)) { // If was on dead server, its closed now. Force to OFFLINE and this // will get it reassigned if appropriate forceOffline(regionInfo, data); } else { - // Just insert region into RIT. + // If this never updates the timeout will trigger new assignment - regionsInTransition.put(encodedRegionName, new RegionState( - regionInfo, RegionState.State.CLOSING, - data.getStamp(), data.getOrigin())); + regionsInTransition.put( + encodedRegionName, + new RegionState(regionInfo, RegionState.State.CLOSING, data + .getStamp(), data.getOrigin())); + sendRegionClose(regionInfo); } break; @@ -1715,7 +1715,7 @@ /** * Unassigns the specified region. *

- * Updates the RegionState and sends the CLOSE RPC. + * Updates the RegionState and creates a zk node. *

* If a RegionPlan is already set, it will remain. * @@ -1765,15 +1765,29 @@ return; } } + sendRegionClose(region); + + } + + /** + * sends the CLOSE RPC to a region server. + * @param region server to be unassigned + */ + public void sendRegionClose(HRegionInfo region) + { // Send CLOSE RPC ServerName server = null; synchronized (this.regions) { server = regions.get(region); + if(null == server){ + LOG.warn("Server is null, region name is " + region.getEncodedName()); + 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 - if (serverManager.sendRegionClose(server, state.getRegion())) { + if (serverManager.sendRegionClose(server, region)) { debugLog(region, "Sent CLOSE to " + server + " for region " + region.getRegionNameAsString()); return; @@ -2527,21 +2541,7 @@ LOG.info("Region has been PENDING_CLOSE for too " + "long, running forced unassign again on region=" + regionInfo.getRegionNameAsString()); - try { - // 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()))) { - // Queue running of an unassign -- do actual unassign - // outside of the regionsInTransition lock. - invokeUnassign(regionInfo); - } - } catch (NoNodeException e) { - LOG.debug("Node no longer existed so not forcing another " - + "unassignment"); - } catch (KeeperException e) { - LOG.warn("Unexpected ZK exception timing out a region close", e); - } + sendRegionClose(regionInfo); break; case CLOSING: LOG.info("Region has been CLOSING for too " + Index: src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java (revision 1201618) +++ src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java (working copy) @@ -378,11 +378,13 @@ // Let's just assign everything to first RS HRegionServer hrs = cluster.getRegionServer(0); ServerName serverName = hrs.getServerName(); - + HRegionInfo closingRegion = enabledRegions.remove(0); // we'll need some regions to already be assigned out properly on live RS List enabledAndAssignedRegions = new ArrayList(); enabledAndAssignedRegions.add(enabledRegions.remove(0)); enabledAndAssignedRegions.add(enabledRegions.remove(0)); + enabledAndAssignedRegions.add(closingRegion); + List disabledAndAssignedRegions = new ArrayList(); disabledAndAssignedRegions.add(disabledRegions.remove(0)); disabledAndAssignedRegions.add(disabledRegions.remove(0)); @@ -436,24 +438,9 @@ /* * ZK = CLOSING */ + regionsThatShouldBeOnline.add(closingRegion); + ZKAssign.createNodeClosing(zkw, closingRegion, serverName); -// Disabled test of CLOSING. This case is invalid after HBASE-3181. -// How can an RS stop a CLOSING w/o deleting the node? If it did ever fail -// and left the node in CLOSING, the RS would have aborted and we'd process -// these regions in server shutdown -// -// // Region of enabled table being closed but not complete -// // Region is already assigned, don't say anything to RS but set ZK closing -// region = enabledAndAssignedRegions.remove(0); -// regionsThatShouldBeOnline.add(region); -// ZKAssign.createNodeClosing(zkw, region, serverName); -// -// // Region of disabled table being closed but not complete -// // Region is already assigned, don't say anything to RS but set ZK closing -// region = disabledAndAssignedRegions.remove(0); -// regionsThatShouldBeOffline.add(region); -// ZKAssign.createNodeClosing(zkw, region, serverName); - /* * ZK = CLOSED */