You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Jieshan Bean (JIRA)" <ji...@apache.org> on 2011/06/02 02:56:47 UTC
[jira] [Created] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
The splitted region can be online again while the standby hmaster becomes the active one
----------------------------------------------------------------------------------------
Key: HBASE-3946
URL: https://issues.apache.org/jira/browse/HBASE-3946
Project: HBase
Issue Type: Bug
Affects Versions: 0.90.3
Reporter: Jieshan Bean
Assignee: Jieshan Bean
Fix For: 0.90.4
(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) {
LOG.info("Master startup proceeding: cluster startup");
this.assignmentManager.cleanoutUnassigned();
this.assignmentManager.assignAllUserRegions();
} 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) {
// Region not being served, add to region map with no assignment
// If this needs to be assigned out, it will also be in ZK as RIT
this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
offlineServers.put(regionLocation, offlineRegions);
}
offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
} else {
// Region is being served and on an active server
regions.put(regionInfo, regionLocation);
addToServers(regionLocation, regionInfo);
}
}
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 {
ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
master.getServerName());
} 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("No regions in transition in ZK to process on failover");
return;
}
LOG.info("Failed-over master needs to process " + nodes.size() +
" regions in transition");
for (String encodedRegionName: nodes) {
processRegionInTransition(encodedRegionName, null);
}
So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
Posted by "Jieshan Bean (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HBASE-3946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Jieshan Bean updated HBASE-3946:
--------------------------------
Attachment: HBASE-3946-V2.patch
Sorry for the prev patch name is wrong.
This patch just modified two places of the code format.
> The splitted region can be online again while the standby hmaster becomes the active one
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-3946
> URL: https://issues.apache.org/jira/browse/HBASE-3946
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.3
> Reporter: Jieshan Bean
> Assignee: Jieshan Bean
> Fix For: 0.90.4
>
> Attachments: HBASE-3926-V2.patch, HBASE-3946-V2.patch, HBASE-3946.patch
>
>
> (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) {
> LOG.info("Master startup proceeding: cluster startup");
> this.assignmentManager.cleanoutUnassigned();
> this.assignmentManager.assignAllUserRegions();
> } 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) {
> // Region not being served, add to region map with no assignment
> // If this needs to be assigned out, it will also be in ZK as RIT
> this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
> offlineServers.put(regionLocation, offlineRegions);
> }
> offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
> } else {
> // Region is being served and on an active server
> regions.put(regionInfo, regionLocation);
> addToServers(regionLocation, regionInfo);
> }
> }
> 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 {
> ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
> master.getServerName());
> } 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("No regions in transition in ZK to process on failover");
> return;
> }
> LOG.info("Failed-over master needs to process " + nodes.size() +
> " regions in transition");
> for (String encodedRegionName: nodes) {
> processRegionInTransition(encodedRegionName, null);
> }
> So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
Posted by "Jieshan Bean (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HBASE-3946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Jieshan Bean updated HBASE-3946:
--------------------------------
Attachment: (was: HBASE-3926-V2.patch)
> The splitted region can be online again while the standby hmaster becomes the active one
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-3946
> URL: https://issues.apache.org/jira/browse/HBASE-3946
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.3
> Reporter: Jieshan Bean
> Assignee: Jieshan Bean
> Fix For: 0.90.4
>
> Attachments: HBASE-3946-V2.patch, HBASE-3946.patch
>
>
> (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) {
> LOG.info("Master startup proceeding: cluster startup");
> this.assignmentManager.cleanoutUnassigned();
> this.assignmentManager.assignAllUserRegions();
> } 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) {
> // Region not being served, add to region map with no assignment
> // If this needs to be assigned out, it will also be in ZK as RIT
> this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
> offlineServers.put(regionLocation, offlineRegions);
> }
> offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
> } else {
> // Region is being served and on an active server
> regions.put(regionInfo, regionLocation);
> addToServers(regionLocation, regionInfo);
> }
> }
> 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 {
> ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
> master.getServerName());
> } 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("No regions in transition in ZK to process on failover");
> return;
> }
> LOG.info("Failed-over master needs to process " + nodes.size() +
> " regions in transition");
> for (String encodedRegionName: nodes) {
> processRegionInTransition(encodedRegionName, null);
> }
> So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
Posted by "Jieshan Bean (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HBASE-3946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Jieshan Bean updated HBASE-3946:
--------------------------------
Attachment: HBASE-3946.patch
> The splitted region can be online again while the standby hmaster becomes the active one
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-3946
> URL: https://issues.apache.org/jira/browse/HBASE-3946
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.3
> Reporter: Jieshan Bean
> Assignee: Jieshan Bean
> Fix For: 0.90.4
>
> Attachments: HBASE-3946.patch
>
>
> (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) {
> LOG.info("Master startup proceeding: cluster startup");
> this.assignmentManager.cleanoutUnassigned();
> this.assignmentManager.assignAllUserRegions();
> } 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) {
> // Region not being served, add to region map with no assignment
> // If this needs to be assigned out, it will also be in ZK as RIT
> this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
> offlineServers.put(regionLocation, offlineRegions);
> }
> offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
> } else {
> // Region is being served and on an active server
> regions.put(regionInfo, regionLocation);
> addToServers(regionLocation, regionInfo);
> }
> }
> 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 {
> ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
> master.getServerName());
> } 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("No regions in transition in ZK to process on failover");
> return;
> }
> LOG.info("Failed-over master needs to process " + nodes.size() +
> " regions in transition");
> for (String encodedRegionName: nodes) {
> processRegionInTransition(encodedRegionName, null);
> }
> So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
Posted by "Jieshan Bean (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HBASE-3946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Jieshan Bean updated HBASE-3946:
--------------------------------
Status: Patch Available (was: Open)
> The splitted region can be online again while the standby hmaster becomes the active one
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-3946
> URL: https://issues.apache.org/jira/browse/HBASE-3946
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.3
> Reporter: Jieshan Bean
> Assignee: Jieshan Bean
> Fix For: 0.90.4
>
> Attachments: HBASE-3946.patch
>
>
> (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) {
> LOG.info("Master startup proceeding: cluster startup");
> this.assignmentManager.cleanoutUnassigned();
> this.assignmentManager.assignAllUserRegions();
> } 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) {
> // Region not being served, add to region map with no assignment
> // If this needs to be assigned out, it will also be in ZK as RIT
> this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
> offlineServers.put(regionLocation, offlineRegions);
> }
> offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
> } else {
> // Region is being served and on an active server
> regions.put(regionInfo, regionLocation);
> addToServers(regionLocation, regionInfo);
> }
> }
> 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 {
> ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
> master.getServerName());
> } 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("No regions in transition in ZK to process on failover");
> return;
> }
> LOG.info("Failed-over master needs to process " + nodes.size() +
> " regions in transition");
> for (String encodedRegionName: nodes) {
> processRegionInTransition(encodedRegionName, null);
> }
> So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
Posted by "Hudson (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HBASE-3946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13050992#comment-13050992 ]
Hudson commented on HBASE-3946:
-------------------------------
Integrated in HBase-TRUNK #1976 (See [https://builds.apache.org/job/HBase-TRUNK/1976/])
> The splitted region can be online again while the standby hmaster becomes the active one
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-3946
> URL: https://issues.apache.org/jira/browse/HBASE-3946
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.3
> Reporter: Jieshan Bean
> Assignee: Jieshan Bean
> Fix For: 0.90.4
>
> Attachments: HBASE-3946-V2.patch, HBASE-3946.patch
>
>
> (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) {
> LOG.info("Master startup proceeding: cluster startup");
> this.assignmentManager.cleanoutUnassigned();
> this.assignmentManager.assignAllUserRegions();
> } 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) {
> // Region not being served, add to region map with no assignment
> // If this needs to be assigned out, it will also be in ZK as RIT
> this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
> offlineServers.put(regionLocation, offlineRegions);
> }
> offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
> } else {
> // Region is being served and on an active server
> regions.put(regionInfo, regionLocation);
> addToServers(regionLocation, regionInfo);
> }
> }
> 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 {
> ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
> master.getServerName());
> } 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("No regions in transition in ZK to process on failover");
> return;
> }
> LOG.info("Failed-over master needs to process " + nodes.size() +
> " regions in transition");
> for (String encodedRegionName: nodes) {
> processRegionInTransition(encodedRegionName, null);
> }
> So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
Posted by "stack (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HBASE-3946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
stack updated HBASE-3946:
-------------------------
Resolution: Fixed
Hadoop Flags: [Reviewed]
Status: Resolved (was: Patch Available)
Committed to branch and trunk. Thanks for the patch Jieshan.
> The splitted region can be online again while the standby hmaster becomes the active one
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-3946
> URL: https://issues.apache.org/jira/browse/HBASE-3946
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.3
> Reporter: Jieshan Bean
> Assignee: Jieshan Bean
> Fix For: 0.90.4
>
> Attachments: HBASE-3946-V2.patch, HBASE-3946.patch
>
>
> (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) {
> LOG.info("Master startup proceeding: cluster startup");
> this.assignmentManager.cleanoutUnassigned();
> this.assignmentManager.assignAllUserRegions();
> } 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) {
> // Region not being served, add to region map with no assignment
> // If this needs to be assigned out, it will also be in ZK as RIT
> this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
> offlineServers.put(regionLocation, offlineRegions);
> }
> offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
> } else {
> // Region is being served and on an active server
> regions.put(regionInfo, regionLocation);
> addToServers(regionLocation, regionInfo);
> }
> }
> 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 {
> ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
> master.getServerName());
> } 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("No regions in transition in ZK to process on failover");
> return;
> }
> LOG.info("Failed-over master needs to process " + nodes.size() +
> " regions in transition");
> for (String encodedRegionName: nodes) {
> processRegionInTransition(encodedRegionName, null);
> }
> So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
Posted by "Jieshan Bean (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HBASE-3946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Jieshan Bean updated HBASE-3946:
--------------------------------
Attachment: HBASE-3926-V2.patch
> The splitted region can be online again while the standby hmaster becomes the active one
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-3946
> URL: https://issues.apache.org/jira/browse/HBASE-3946
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.3
> Reporter: Jieshan Bean
> Assignee: Jieshan Bean
> Fix For: 0.90.4
>
> Attachments: HBASE-3926-V2.patch, HBASE-3946.patch
>
>
> (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) {
> LOG.info("Master startup proceeding: cluster startup");
> this.assignmentManager.cleanoutUnassigned();
> this.assignmentManager.assignAllUserRegions();
> } 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) {
> // Region not being served, add to region map with no assignment
> // If this needs to be assigned out, it will also be in ZK as RIT
> this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
> offlineServers.put(regionLocation, offlineRegions);
> }
> offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
> } else {
> // Region is being served and on an active server
> regions.put(regionInfo, regionLocation);
> addToServers(regionLocation, regionInfo);
> }
> }
> 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 {
> ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
> master.getServerName());
> } 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("No regions in transition in ZK to process on failover");
> return;
> }
> LOG.info("Failed-over master needs to process " + nodes.size() +
> " regions in transition");
> for (String encodedRegionName: nodes) {
> processRegionInTransition(encodedRegionName, null);
> }
> So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
Posted by "Hudson (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HBASE-3946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13056860#comment-13056860 ]
Hudson commented on HBASE-3946:
-------------------------------
Integrated in HBase-TRUNK #1995 (See [https://builds.apache.org/job/HBase-TRUNK/1995/])
> The splitted region can be online again while the standby hmaster becomes the active one
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-3946
> URL: https://issues.apache.org/jira/browse/HBASE-3946
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.3
> Reporter: Jieshan Bean
> Assignee: Jieshan Bean
> Fix For: 0.90.4
>
> Attachments: HBASE-3946-V2.patch, HBASE-3946.patch
>
>
> (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) {
> LOG.info("Master startup proceeding: cluster startup");
> this.assignmentManager.cleanoutUnassigned();
> this.assignmentManager.assignAllUserRegions();
> } 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) {
> // Region not being served, add to region map with no assignment
> // If this needs to be assigned out, it will also be in ZK as RIT
> this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
> offlineServers.put(regionLocation, offlineRegions);
> }
> offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
> } else {
> // Region is being served and on an active server
> regions.put(regionInfo, regionLocation);
> addToServers(regionLocation, regionInfo);
> }
> }
> 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 {
> ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
> master.getServerName());
> } 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("No regions in transition in ZK to process on failover");
> return;
> }
> LOG.info("Failed-over master needs to process " + nodes.size() +
> " regions in transition");
> for (String encodedRegionName: nodes) {
> processRegionInTransition(encodedRegionName, null);
> }
> So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
Posted by "Jieshan Bean (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HBASE-3946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13042570#comment-13042570 ]
Jieshan Bean commented on HBASE-3946:
-------------------------------------
For the patch I haven't take enough test on it. But I can describe my solution:
{noformat}
Index: src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
===================================================================
--- src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (revision 1130364)
+++ src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (working copy)
@@ -1470,14 +1470,17 @@
Result result = region.getSecond();
// If region was in transition (was in zk) force it offline for reassign
try {
- ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
- master.getServerName());
+ //Process with existing RS shutdown code
+ boolean isNotDisabledAndSplitted =
+ ServerShutdownHandler.processDeadRegion(regionInfo, result, this,
+ this.catalogTracker);
+ if (isNotDisabledAndSplitted) {
+ ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
+ master.getServerName());
+ }
} catch (KeeperException.NoNodeException nne) {
// This is fine
}
- // Process with existing RS shutdown code
- ServerShutdownHandler.processDeadRegion(regionInfo, result, this,
- this.catalogTracker);
}
}
}
{noformat}
> The splitted region can be online again while the standby hmaster becomes the active one
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-3946
> URL: https://issues.apache.org/jira/browse/HBASE-3946
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.3
> Reporter: Jieshan Bean
> Assignee: Jieshan Bean
> Fix For: 0.90.4
>
>
> (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) {
> LOG.info("Master startup proceeding: cluster startup");
> this.assignmentManager.cleanoutUnassigned();
> this.assignmentManager.assignAllUserRegions();
> } 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) {
> // Region not being served, add to region map with no assignment
> // If this needs to be assigned out, it will also be in ZK as RIT
> this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
> offlineServers.put(regionLocation, offlineRegions);
> }
> offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
> } else {
> // Region is being served and on an active server
> regions.put(regionInfo, regionLocation);
> addToServers(regionLocation, regionInfo);
> }
> }
> 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 {
> ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
> master.getServerName());
> } 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("No regions in transition in ZK to process on failover");
> return;
> }
> LOG.info("Failed-over master needs to process " + nodes.size() +
> " regions in transition");
> for (String encodedRegionName: nodes) {
> processRegionInTransition(encodedRegionName, null);
> }
> So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HBASE-3946) The splitted region can be online
again while the standby hmaster becomes the active one
Posted by "Jieshan Bean (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HBASE-3946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13043199#comment-13043199 ]
Jieshan Bean commented on HBASE-3946:
-------------------------------------
I have taken some tests on this patch, and it indeed works.
Stack, can you give out your suggestions on this patch? Thanks!
> The splitted region can be online again while the standby hmaster becomes the active one
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-3946
> URL: https://issues.apache.org/jira/browse/HBASE-3946
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.3
> Reporter: Jieshan Bean
> Assignee: Jieshan Bean
> Fix For: 0.90.4
>
> Attachments: HBASE-3946.patch
>
>
> (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) {
> LOG.info("Master startup proceeding: cluster startup");
> this.assignmentManager.cleanoutUnassigned();
> this.assignmentManager.assignAllUserRegions();
> } 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) {
> // Region not being served, add to region map with no assignment
> // If this needs to be assigned out, it will also be in ZK as RIT
> this.regions.put(regionInfo, 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 = new ArrayList<Pair<HRegionInfo,Result>>(1);
> offlineServers.put(regionLocation, offlineRegions);
> }
> offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
> } else {
> // Region is being served and on an active server
> regions.put(regionInfo, regionLocation);
> addToServers(regionLocation, regionInfo);
> }
> }
> 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 {
> ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
> master.getServerName());
> } 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("No regions in transition in ZK to process on failover");
> return;
> }
> LOG.info("Failed-over master needs to process " + nodes.size() +
> " regions in transition");
> for (String encodedRegionName: nodes) {
> processRegionInTransition(encodedRegionName, null);
> }
> So I think before add the region into RIT, check it at first.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira