You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Gregory Chanan (JIRA)" <ji...@apache.org> on 2012/10/20 00:40:12 UTC

[jira] [Updated] (HBASE-7018) Fix and Improve TableDescriptor caching for bulk assignment

     [ https://issues.apache.org/jira/browse/HBASE-7018?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Gregory Chanan updated HBASE-7018:
----------------------------------

    Status: Patch Available  (was: Open)
    
> Fix and Improve TableDescriptor caching for bulk assignment
> -----------------------------------------------------------
>
>                 Key: HBASE-7018
>                 URL: https://issues.apache.org/jira/browse/HBASE-7018
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver
>            Reporter: Gregory Chanan
>            Assignee: Gregory Chanan
>             Fix For: 0.94.3, 0.96.0
>
>         Attachments: HBASE-7018-94.patch
>
>
> HBASE-6214 backported HBASE-5998 (Bulk assignment: regionserver optimization by using a temporary cache for table descriptors when receiving an open regions request), but it's buggy on 0.94 (0.96 appears correct):
> {code}
>     HTableDescriptor htd = null;
>     if (htds == null) {
>       htd = this.tableDescriptors.get(region.getTableName());
>     } else {
>       htd = htds.get(region.getTableNameAsString());
>       if (htd == null) {
>         htd = this.tableDescriptors.get(region.getTableName());
>         htds.put(region.getRegionNameAsString(), htd);
>       }
>     }
> {code}
> i.e. we get the tableName from the map but write the regionName.
> Even fixing this, it looks like there are areas for improvement:
> 1) FSTableDescriptors already has a cache (though it goes to the NameNode each time through to check we have the latest copy.  May as well combine these two caches, might be a performance win as well since we don't need to write to multiple caches.
> 2) FSTableDescriptors makes two RPCs to the NameNode when it encounters a new table.  So the total number of RPCs necessary for a bulk assign (without caching is):
> #regions + #tables
> (with caching):
> min(#regions,#tables) + #tables = #tables + #tables = 2 * #tables
> We can make this only one RPC, yielding:
> #tables
> Probably not a big deal for most users, but in a multi-tenant situation where the number of regions being bulk assigned approaches the number of tables being bulk assigned, this could be a nice performance win.
> Benchmarks coming.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira