[ 
https://issues.apache.org/jira/browse/HBASE-6060?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13287561#comment-13287561
 ] 

Zhihong Yu commented on HBASE-6060:
-----------------------------------

I ran the tests in TestAssignmentManager and they passed.
{code}
     synchronized (this.regionPlans) {
+      regionsOnDeadServer = new RegionsOnDeadServer();
+      regionsFromRegionPlansForServer = new 
ConcurrentSkipListSet<HRegionInfo>();
+      this.deadServerRegionsFromRegionPlan.put(sn, regionsOnDeadServer);
{code}
Can the first two assignments be placed outside synchronized block ?
Before making the deadServerRegionsFromRegionPlan.put() call, I think we should 
check that sn isn't currently in deadServerRegionsFromRegionPlan.
For isRegionOnline(HRegionInfo hri):
{code}
+        return true;
+      } else {
+        // Remove the assignment mapping for sn.
+        Set<HRegionInfo> hriSet = this.servers.get(sn);
+        if (hriSet != null) {
+          hriSet.remove(hri);
+        }
{code}
The else keyword isn't needed.
What if hriSet contains other regions apart from hri, should they be removed as 
well ?
                
> Regions's in OPENING state from failed regionservers takes a long time to 
> recover
> ---------------------------------------------------------------------------------
>
>                 Key: HBASE-6060
>                 URL: https://issues.apache.org/jira/browse/HBASE-6060
>             Project: HBase
>          Issue Type: Bug
>          Components: master, regionserver
>            Reporter: Enis Soztutar
>            Assignee: Enis Soztutar
>         Attachments: HBASE-6060-94.patch
>
>
> we have seen a pattern in tests, that the regions are stuck in OPENING state 
> for a very long time when the region server who is opening the region fails. 
> My understanding of the process: 
>  
>  - master calls rs to open the region. If rs is offline, a new plan is 
> generated (a new rs is chosen). RegionState is set to PENDING_OPEN (only in 
> master memory, zk still shows OFFLINE). See HRegionServer.openRegion(), 
> HMaster.assign()
>  - RegionServer, starts opening a region, changes the state in znode. But 
> that znode is not ephemeral. (see ZkAssign)
>  - Rs transitions zk node from OFFLINE to OPENING. See 
> OpenRegionHandler.process()
>  - rs then opens the region, and changes znode from OPENING to OPENED
>  - when rs is killed between OPENING and OPENED states, then zk shows OPENING 
> state, and the master just waits for rs to change the region state, but since 
> rs is down, that wont happen. 
>  - There is a AssignmentManager.TimeoutMonitor, which does exactly guard 
> against these kind of conditions. It periodically checks (every 10 sec by 
> default) the regions in transition to see whether they timedout 
> (hbase.master.assignment.timeoutmonitor.timeout). Default timeout is 30 min, 
> which explains what you and I are seeing. 
>  - ServerShutdownHandler in Master does not reassign regions in OPENING 
> state, although it handles other states. 
> Lowering that threshold from the configuration is one option, but still I 
> think we can do better. 
> Will investigate more. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to