You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Pankaj Kumar (Jira)" <ji...@apache.org> on 2020/07/30 12:27:00 UTC

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

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

Pankaj Kumar updated HBASE-24800:
---------------------------------
    Description: 
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 use ZK Multi to enhance this.

  was:
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. 


> 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 use ZK Multi to enhance this.



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