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

ramkrishna.s.vasudevan commented on HBASE-6060:
-----------------------------------------------

Thanks Stack.
Just to give more details so that you can review with these points also in mind.
When SSH starts processing
{code}
        this.services.getAssignmentManager().
          processServerShutdown(this.serverName);
{code}
{code}
     synchronized (this.regionPlans) {
+      this.deadServerRegionsFromRegionPlan.put(sn, regionsOnDeadServer);
       for (Iterator <Map.Entry<String, RegionPlan>> i =
           this.regionPlans.entrySet().iterator(); i.hasNext();) {
         Map.Entry<String, RegionPlan> e = i.next();
@@ -3210,9 +3265,11 @@
         // The name will be null if the region is planned for a random assign.
         if (otherSn != null && otherSn.equals(sn)) {
           // Use iterator's remove else we'll get CME
+          regionsFromRegionPlansForServer.add(e.getValue().getRegionInfo());
           i.remove();
         }
       }
+      
regionsOnDeadServer.setRegionsFromRegionPlansForServer(regionsFromRegionPlansForServer);
{code}
If the SSH first sees that the region plan was part of the destination that had 
gone down, as per the patch we add the region plan into the new datastructure.  

This time the getREgionPlan will be blocked as it is synchronzied.  
Now after this step when getRegionPlan proceeds we again synchronize on 
regionplans
{code}
synchronized (this.regionPlans) {
.....
.....
+      if (serverToExclude != null) {
+        RegionsOnDeadServer regionsOnDeadServer = 
this.deadServerRegionsFromRegionPlan
+            .get(serverToExclude);
+        if (regionsOnDeadServer != null
+            && regionsOnDeadServer.getRegionsFromRegionPlansForServer().
+            contains(state.getRegion())) {
+          if (newPlan) {
+            this.regionPlans.remove(randomPlan.getRegionName());
+            LOG
+            .info("Server shutdown handler already in progress for the region "
+                + randomPlan.getRegionName());
+            randomPlan = RegionPlan.REGION_PLAN_ALREADY_INUSE;
+          } else {
+            this.regionPlans.remove(existingPlan.getRegionName());
+            LOG
+            .info("Server shutdown handler already in progress for the region "
+                + existingPlan.getRegionName());
+            existingPlan = RegionPlan.REGION_PLAN_ALREADY_INUSE;
+          }
+        }
+      }
{code}
Now if the getRegionPlan will see that already SSH populated the datastructure 
and so he will say do not use this plan. And hence the assign does not happen.
Next if suppose getRegionPlan happens first, any way he will populate a new 
region plan and add it into the region plan and for sure it will not be the 
server that has gone down becuase it will not be in onlineServerlist.
Now after this if SSH processServerShutdown starts, when he starts to iterate 
the region Plan he will not find the destination server as the dead server and 
hence SSH need  not bother about it.

bq.(we could add another field to RegionPlan with where it is in the assignment 
process?)
In our first patch we have used some boolean flag to say if to use the plan or 
not. Later changed it to special states.






                
> 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
>             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, 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