You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Anoop Sam John (Jira)" <ji...@apache.org> on 2020/08/03 03:49:00 UTC

[jira] [Commented] (HBASE-24800) Enhance ACL region initialization

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

Anoop Sam John commented on HBASE-24800:
----------------------------------------

60K table? !
So here zk level call, optimization is proposed. I have 2 other aspects to mention.
- We use zk as a way to notify all RSs to update their cache of the table permission details when the ACL table is updated. But we end up writing all ACL table detail into zk also. With these many tables like cases, its lot of data in the zk.  Should we think of a way where we can just inform about the ACL content change to RSs and each RS read the latest changed content from ACL region in order to update its cache? This read can be time range based if every RS track the latest TS of its local ACL cache content.
- Per RS we keep the ACL detail cached in RS. How big this is growing in ur cases with these many tables? Per RS you might have regions from so many tables also right?  Even for the local cache we need some cap for the heap usage?

> Enhance ACL region initialization
> ---------------------------------
>
>                 Key: HBASE-24800
>                 URL: https://issues.apache.org/jira/browse/HBASE-24800
>             Project: HBase
>          Issue Type: Improvement
>          Components: acl, MTTR
>            Reporter: Pankaj Kumar
>            Assignee: Pankaj Kumar
>            Priority: Major
>
> RegionServer persist ACL table entries into Zookeeper during ACL region open,
> {code}
>       private void initialize(RegionCoprocessorEnvironment e) throws IOException {
>     final Region region = e.getRegion();
>     Configuration conf = e.getConfiguration();
>     Map<byte[], ListMultimap<String, UserPermission>> tables = PermissionStorage.loadAll(region);
>     // For each table, write out the table's permissions to the respective
>     // znode for that table.
>     for (Map.Entry<byte[], ListMultimap<String, UserPermission>> t:
>       tables.entrySet()) {
>       byte[] entry = t.getKey();
>       ListMultimap<String, UserPermission> perms = t.getValue();
>       byte[] serialized = PermissionStorage.writePermissionsAsBytes(perms, conf);
>       zkPermissionWatcher.writeToZookeeper(entry, serialized);
>     }
>     initialized = true;
>   }
> {code}
> Currently RegionServer send 2 RPC (one to create the table path and another to set the data) for each table sequentially.
> {code}
>  try {
>       ZKUtil.createWithParents(watcher, zkNode);
>       ZKUtil.updateExistingNodeData(watcher, zkNode, permsData, -1);
>     } catch (KeeperException e) {
>       LOG.error("Failed updating permissions for entry '" +
>           entryName + "'", e);
>       watcher.abort("Failed writing node "+zkNode+" to zookeeper", e);
>     }
> {code}
> If a cluster have huge number of tables then ACL region open will take time. Example, it took ~9 min to write 60k tables ACL into ZK. 
> We should send ZK Ops in a single multi() to enhance this.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)