[
https://issues.apache.org/jira/browse/HBASE-5970?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13286342#comment-13286342
]
chunhui shen commented on HBASE-5970:
-------------------------------------
I run the failed test
case(org.apache.hadoop.hbase.master.TestAssignmentManager) and passed on the
local PC.
Also I think I found the reason why it failed: caused by
TestAssignmentManager#testRegionPlanIsUpdatedWhenRegionFailsToOpen
At the last of testRegionPlanIsUpdatedWhenRegionFailsToOpen, am will handle
region with the state RS_ZK_REGION_FAILED_OPEN,
in the AssignmentManager#handleRegion()
{code}
case RS_ZK_REGION_FAILED_OPEN:
...
this.executorService.submit(new ClosedRegionHandler(master,
this, regionState.getRegion()));
{code}
so we call a thread to execute ClosedRegionHandler in the background, and it
create a node after
testRegionPlanIsUpdatedWhenRegionFailsToOpen execute
{code}public void after() throws KeeperException {
if (this.watcher != null) {
// Clean up all znodes
ZKAssign.deleteAllNodes(this.watcher);
this.watcher.close();
}
}{code}
Hence, it will fail with a probability and nothing with this patch.
> 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
> Priority: Critical
> Attachments: 5970v3.patch, HBASE-5970.patch, HBASE-5970v2.patch,
> HBASE-5970v3.patch, HBASE-5970v4.patch, HBASE-5970v4.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