Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
0.90.3
-
None
-
None
-
Reviewed
Description
(The cluster has two HMatser, one active and one standby)
1.While the active HMaster shutdown, the standby one would become the active one, and went into the processFailover() method:
if (regionCount == 0)
else
{ LOG.info("Master startup proceeding: master failover"); this.assignmentManager.processFailover(); }2.After that, the user regions would be rebuild.
Map<HServerInfo,List<Pair<HRegionInfo,Result>>> deadServers = rebuildUserRegions();
3.Here's how the rebuildUserRegions worked. All the regions(contain the splitted regions) would be added to the offlineRegions of offlineServers.
for (Result result : results) {
Pair<HRegionInfo,HServerInfo> region =
MetaReader.metaRowToRegionPairWithInfo(result);
if (region == null) continue;
HServerInfo regionLocation = region.getSecond();
HRegionInfo regionInfo = region.getFirst();
if (regionLocation == null)
else if (!serverManager.isServerOnline(
regionLocation.getServerName())) {
// Region is located on a server that isn't online
List<Pair<HRegionInfo,Result>> offlineRegions =
offlineServers.get(regionLocation);
if (offlineRegions == null)
offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
} else
}
4.It seems that all the offline regions will be added to RIT and online again:
ZKAssign will creat node for each offline never consider the splitted ones.
AssignmentManager# processDeadServers
private void processDeadServers(
Map<HServerInfo, List<Pair<HRegionInfo, Result>>> deadServers)
throws IOException, KeeperException {
for (Map.Entry<HServerInfo, List<Pair<HRegionInfo,Result>>> deadServer :
deadServers.entrySet()) {
List<Pair<HRegionInfo,Result>> regions = deadServer.getValue();
for (Pair<HRegionInfo,Result> region : regions) {
HRegionInfo regionInfo = region.getFirst();
Result result = region.getSecond();
// If region was in transition (was in zk) force it offline for reassign
try
catch (KeeperException.NoNodeException nne)
{ // This is fine } // Process with existing RS shutdown code
ServerShutdownHandler.processDeadRegion(regionInfo, result, this,
this.catalogTracker);
}
}
}
AssignmentManager# processFailover
// Process list of dead servers
processDeadServers(deadServers);
// Check existing regions in transition
List<String> nodes = ZKUtil.listChildrenAndWatchForNewChildren(watcher,
watcher.assignmentZNode);
if (nodes.isEmpty())
LOG.info("Failed-over master needs to process " + nodes.size() +
" regions in transition");
for (String encodedRegionName: nodes)
So I think before add the region into RIT, check it at first.