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

stack commented on HBASE-6060:
------------------------------

What do I have to change in that section of code?  It seems fine as is.  Or are 
you thinking we need to allow it being in OFFLINE state?

Yes, agree that before setting PENDING_OPEN, need to ensure it has not moved to 
OPENING.  That should be easy enough (in fact, its not possible now I think 
about it since there a lock on RegionState that surrounds the single-assign -- 
need to make sure handleRegion respects it).

What do you think of this approach Ram?  The one where we rely on RegionState 
rather than on RegionPlan?

I won't have the issue you see in Rajesh's patch, right?  I don't intend to 
collect regions from outstanding regionplans.  If the RS goes down before 
sending the RPC, then the single-assign will fail and retry but it won't be 
included in the SSH set? 


                
> 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: rajeshbabu
>             Fix For: 0.96.0, 0.94.1, 0.92.3
>
>         Attachments: 6060-94-v3.patch, 6060-94-v4.patch, 6060-94-v4_1.patch, 
> 6060-94-v4_1.patch, 6060-trunk.patch, 6060-trunk.patch, 6060-trunk_2.patch, 
> 6060-trunk_3.patch, 6060_alternative_suggestion.txt, 
> 6060_suggestion2_based_off_v3.patch, 6060_suggestion_based_off_v3.patch, 
> 6060_suggestion_toassign_rs_wentdown_beforerequest.patch, 
> HBASE-6060-92.patch, 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