hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ramkrishna.s.vasudevan (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-5970) Improve the AssignmentManager#updateTimer and speed up handling opened event
Date Wed, 09 May 2012 05:16:57 GMT

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

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

@Chunhui
HBASE-5337-AM.updateTimers() delays the timeout monitor from assigning regions.
I think i can duplicate it with this defect.
Few comments
serversInUpdatingTimer can be changed to updateTimerForServer?
The order in which the updation should happen matters? 
{code}
Threads.setDaemonThreadRunning(timerUpdater.getThread(),
+        master.getServerName() + ".timerUpdater");
{code}
Can we just add this to one protected method.  Actually we found that doing like that will
help you in mocking the master in case of testing these scenarios?

Patch looks good.


                
> 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: HBASE-5970.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 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

        

Mime
View raw message