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

chunhui shen commented on HBASE-5970:
-------------------------------------

bq.Why is doing this work in the background faster? Is it just that it being 
inline, it takes a noticeable amount of time?

AssignmentManager#updateTimers would check each region in RIT, and there are 
lots of regions in the RIT when startup, so it will took much time(about 30ms 
if 100k regions in RIT). However, in the current logic, we will do the 
AssignmentManager#updateTimers for each opened event, causing the whole process 
of handling all the opened events tooks much much time.

If we do the updateTimers in the background, we needn't wait it when handling 
opened event. Also, we wouldn't updateTimers for the same Regionserver in a 
short time.(In fact, when cluster startup , lots of opened events from the same 
regionserver at the same time, we needn't do this work many times at the moment)
                
> Improve the AssignmentManager#updateTimer and speed up handling opened event
> ----------------------------------------------------------------------------
>
>                 Key: HBASE-5970
>                 URL: https://issues.apache.org/jira/browse/HBASE-5970
>             Project: HBase
>          Issue Type: Improvement
>          Components: master
>            Reporter: chunhui shen
>            Assignee: chunhui shen
>         Attachments: 5970v3.patch, HBASE-5970.patch, HBASE-5970v2.patch, 
> HBASE-5970v3.patch
>
>
> We found handing opened event very slow in the environment with lots of 
> regions.
> The problem is the slow AssignmentManager#updateTimer.
> We do the test for bulk assigning 10w (i.e. 100k) regions, the whole process 
> of bulk assigning took 1 hours.
> 2012-05-06 20:31:49,201 INFO 
> org.apache.hadoop.hbase.master.AssignmentManager: Bulk assigning 100000 
> region(s) round-robin across 5 server(s)
> 2012-05-06 21:26:32,103 INFO 
> org.apache.hadoop.hbase.master.AssignmentManager: Bulk assigning done
> I think we could do the improvement for the AssignmentManager#updateTimer: 
> Make a thread do this work.
> After the improvement, it took only 4.5mins
> 2012-05-07 11:03:36,581 INFO 
> org.apache.hadoop.hbase.master.AssignmentManager: Bulk assigning 100000 
> region(s) across 5 server(s), retainAssignment=true 
> 2012-05-07 11:07:57,073 INFO 
> org.apache.hadoop.hbase.master.AssignmentManager: Bulk assigning done 

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