You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/01/10 03:28:59 UTC

[04/42] hbase git commit: HBASE-19483 Add proper privilege check for rsgroup commands

HBASE-19483 Add proper privilege check for rsgroup commands

Signed-off-by: tedyu <yu...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3fa3dcd9
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3fa3dcd9
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3fa3dcd9

Branch: refs/heads/HBASE-19397-branch-2
Commit: 3fa3dcd9f92e67e74c9ad197d87460c4eb5118ba
Parents: 895267d
Author: Guangxu Cheng <gu...@gmail.com>
Authored: Tue Jan 9 17:56:32 2018 +0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Jan 9 08:11:07 2018 -0800

----------------------------------------------------------------------
 ...tegrationTestIngestWithVisibilityLabels.java |   7 +-
 ...egrationTestBigLinkedListWithVisibility.java |   7 +-
 ...tionTestWithCellVisibilityLoadAndVerify.java |   7 +-
 .../TestImportTSVWithVisibilityLabels.java      |   6 +-
 .../hbase/rest/TestScannersWithLabels.java      |   6 +-
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     | 134 ++--
 .../hbase/rsgroup/TestRSGroupsWithACL.java      | 329 +++++++++
 .../hadoop/hbase/master/MasterRpcServices.java  |   3 +-
 .../hbase/security/access/AccessChecker.java    | 332 +++++++++
 .../security/access/AccessControlLists.java     |   4 +-
 .../hbase/security/access/AccessController.java | 697 ++++++------------
 .../hbase/security/access/TableAuthManager.java |   2 +-
 .../visibility/VisibilityController.java        |  13 +-
 .../hbase/security/access/SecureTestUtil.java   |   6 +
 .../security/access/TestAccessController.java   |  75 --
 .../security/visibility/VisibilityTestUtil.java |   3 +-
 ...TestThriftHBaseServiceHandlerWithLabels.java | 706 +++++++++----------
 .../asciidoc/_chapters/appendix_acl_matrix.adoc |  11 +
 src/main/asciidoc/_chapters/ops_mgt.adoc        |  10 +
 src/main/asciidoc/_chapters/security.adoc       |  36 +
 20 files changed, 1426 insertions(+), 968 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3fa3dcd9/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithVisibilityLabels.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithVisibilityLabels.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithVisibilityLabels.java
index b7d8dad..2928b6d 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithVisibilityLabels.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithVisibilityLabels.java
@@ -23,11 +23,10 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.LoadTestDataGeneratorWithVisibilityLabels;
 import org.apache.hadoop.hbase.security.visibility.VisibilityClient;
-import org.apache.hadoop.hbase.security.visibility.VisibilityController;
+import org.apache.hadoop.hbase.security.visibility.VisibilityTestUtil;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.LoadTestTool;
 import org.junit.experimental.categories.Category;
@@ -76,9 +75,7 @@ public class IntegrationTestIngestWithVisibilityLabels extends IntegrationTestIn
   public void setUpCluster() throws Exception {
     util = getTestingUtil(null);
     Configuration conf = util.getConfiguration();
-    conf.setInt(HFile.FORMAT_VERSION_KEY, 3);
-    conf.set("hbase.coprocessor.master.classes", VisibilityController.class.getName());
-    conf.set("hbase.coprocessor.region.classes", VisibilityController.class.getName());
+    VisibilityTestUtil.enableVisiblityLabels(conf);
     conf.set("hbase.superuser", "admin," + User.getCurrent().getName());
     super.setUpCluster();
     addLabels();

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fa3dcd9/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
index d0e6e52..9349f7b 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.mapreduce.Import;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
@@ -57,7 +56,7 @@ import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.visibility.Authorizations;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.security.visibility.VisibilityClient;
-import org.apache.hadoop.hbase.security.visibility.VisibilityController;
+import org.apache.hadoop.hbase.security.visibility.VisibilityTestUtil;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -373,9 +372,7 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
   public void setUpCluster() throws Exception {
     util = getTestingUtil(null);
     Configuration conf = util.getConfiguration();
-    conf.setInt(HFile.FORMAT_VERSION_KEY, 3);
-    conf.set("hbase.coprocessor.master.classes", VisibilityController.class.getName());
-    conf.set("hbase.coprocessor.region.classes", VisibilityController.class.getName());
+    VisibilityTestUtil.enableVisiblityLabels(conf);
     conf.set("hbase.superuser", User.getCurrent().getName());
     conf.setBoolean("dfs.permissions", false);
     USER = User.createUserForTesting(conf, userName, new String[] {});

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fa3dcd9/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
index 3cafe9d..a9c24e9 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.ScannerCallable;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.mapreduce.TableMapper;
 import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl;
@@ -48,7 +47,7 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.Authorizations;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.security.visibility.VisibilityClient;
-import org.apache.hadoop.hbase.security.visibility.VisibilityController;
+import org.apache.hadoop.hbase.security.visibility.VisibilityTestUtil;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -116,9 +115,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT
   public void setUpCluster() throws Exception {
     util = getTestingUtil(null);
     Configuration conf = util.getConfiguration();
-    conf.setInt(HFile.FORMAT_VERSION_KEY, 3);
-    conf.set("hbase.coprocessor.master.classes", VisibilityController.class.getName());
-    conf.set("hbase.coprocessor.region.classes", VisibilityController.class.getName());
+    VisibilityTestUtil.enableVisiblityLabels(conf);
     conf.set("hbase.superuser", User.getCurrent().getName());
     conf.setBoolean("dfs.permissions", false);
     super.setUpCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fa3dcd9/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
index 8d3f3df..6d6b729 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
@@ -58,7 +58,7 @@ import org.apache.hadoop.hbase.security.visibility.ScanLabelGenerator;
 import org.apache.hadoop.hbase.security.visibility.SimpleScanLabelGenerator;
 import org.apache.hadoop.hbase.security.visibility.VisibilityClient;
 import org.apache.hadoop.hbase.security.visibility.VisibilityConstants;
-import org.apache.hadoop.hbase.security.visibility.VisibilityController;
+import org.apache.hadoop.hbase.security.visibility.VisibilityTestUtil;
 import org.apache.hadoop.hbase.security.visibility.VisibilityUtils;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
@@ -121,9 +121,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
     conf = util.getConfiguration();
     SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
     conf.set("hbase.superuser", "admin,"+User.getCurrent().getName());
-    conf.setInt("hfile.format.version", 3);
-    conf.set("hbase.coprocessor.master.classes", VisibilityController.class.getName());
-    conf.set("hbase.coprocessor.region.classes", VisibilityController.class.getName());
+    VisibilityTestUtil.enableVisiblityLabels(conf);
     conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class,
         ScanLabelGenerator.class);
     util.startMiniCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fa3dcd9/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java
index 6ac8e87..8d738ed 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.security.visibility.ScanLabelGenerator;
 import org.apache.hadoop.hbase.security.visibility.SimpleScanLabelGenerator;
 import org.apache.hadoop.hbase.security.visibility.VisibilityClient;
 import org.apache.hadoop.hbase.security.visibility.VisibilityConstants;
-import org.apache.hadoop.hbase.security.visibility.VisibilityController;
+import org.apache.hadoop.hbase.security.visibility.VisibilityTestUtil;
 import org.apache.hadoop.hbase.security.visibility.VisibilityUtils;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RestTests;
@@ -130,10 +130,8 @@ public class TestScannersWithLabels {
     conf = TEST_UTIL.getConfiguration();
     conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS,
         SimpleScanLabelGenerator.class, ScanLabelGenerator.class);
-    conf.setInt("hfile.format.version", 3);
     conf.set("hbase.superuser", SUPERUSER.getShortName());
-    conf.set("hbase.coprocessor.master.classes", VisibilityController.class.getName());
-    conf.set("hbase.coprocessor.region.classes", VisibilityController.class.getName());
+    VisibilityTestUtil.enableVisiblityLabels(conf);
     TEST_UTIL.startMiniCluster(1);
     // Wait for the labels table to become available
     TEST_UTIL.waitTableEnabled(VisibilityConstants.LABELS_TABLE_NAME.getName(), 50000);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fa3dcd9/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index 2af6c03..fd632b1 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.hbase.rsgroup;
 
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashSet;
@@ -26,10 +31,6 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
-
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
+import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -75,10 +77,16 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGro
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveServersResponse;
 import org.apache.hadoop.hbase.protobuf.generated.TableProtos;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.access.AccessChecker;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.security.access.TableAuthManager;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
 // TODO: Encapsulate MasterObserver functions into separate subclass.
 @CoreCoprocessor
@@ -92,12 +100,17 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
   private RSGroupInfoManager groupInfoManager;
   private RSGroupAdminServer groupAdminServer;
   private final RSGroupAdminService groupAdminService = new RSGroupAdminServiceImpl();
+  private AccessChecker accessChecker;
+
+  /** Provider for mapping principal names to Users */
+  private UserProvider userProvider;
 
   @Override
   public void start(CoprocessorEnvironment env) throws IOException {
     if (!(env instanceof HasMasterServices)) {
       throw new IOException("Does not implement HMasterServices");
     }
+
     master = ((HasMasterServices)env).getMasterServices();
     groupInfoManager = RSGroupInfoManagerImpl.getInstance(master);
     groupAdminServer = new RSGroupAdminServer(master, groupInfoManager);
@@ -106,6 +119,16 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
     if (!RSGroupableBalancer.class.isAssignableFrom(clazz)) {
       throw new IOException("Configured balancer does not support RegionServer groups.");
     }
+    ZKWatcher zk = ((HasMasterServices)env).getMasterServices().getZooKeeper();
+    accessChecker = new AccessChecker(env.getConfiguration(), zk);
+
+    // set the user-provider.
+    this.userProvider = UserProvider.instantiate(env.getConfiguration());
+  }
+
+  @Override
+  public void stop(CoprocessorEnvironment env) {
+    TableAuthManager.release(accessChecker.getAuthManager());
   }
 
   @Override
@@ -137,6 +160,7 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, group="
               + groupName);
       try {
+        checkPermission("getRSGroupInfo");
         RSGroupInfo rsGroupInfo = groupAdminServer.getRSGroupInfo(groupName);
         if (rsGroupInfo != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(rsGroupInfo));
@@ -151,10 +175,11 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
     public void getRSGroupInfoOfTable(RpcController controller,
         GetRSGroupInfoOfTableRequest request, RpcCallback<GetRSGroupInfoOfTableResponse> done) {
       GetRSGroupInfoOfTableResponse.Builder builder = GetRSGroupInfoOfTableResponse.newBuilder();
+      TableName tableName = ProtobufUtil.toTableName(request.getTableName());
+      LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, table="
+          + tableName);
       try {
-        TableName tableName = ProtobufUtil.toTableName(request.getTableName());
-        LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, table="
-                + tableName);
+        checkPermission("getRSGroupInfoOfTable");
         RSGroupInfo RSGroupInfo = groupAdminServer.getRSGroupInfoOfTable(tableName);
         if (RSGroupInfo != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
@@ -169,13 +194,14 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
     public void moveServers(RpcController controller, MoveServersRequest request,
         RpcCallback<MoveServersResponse> done) {
       MoveServersResponse.Builder builder = MoveServersResponse.newBuilder();
+      Set<Address> hostPorts = Sets.newHashSet();
+      for (HBaseProtos.ServerName el : request.getServersList()) {
+        hostPorts.add(Address.fromParts(el.getHostName(), el.getPort()));
+      }
+      LOG.info(master.getClientIdAuditPrefix() + " move servers " + hostPorts +" to rsgroup "
+          + request.getTargetGroup());
       try {
-        Set<Address> hostPorts = Sets.newHashSet();
-        for (HBaseProtos.ServerName el : request.getServersList()) {
-          hostPorts.add(Address.fromParts(el.getHostName(), el.getPort()));
-        }
-        LOG.info(master.getClientIdAuditPrefix() + " move servers " + hostPorts +" to rsgroup "
-                + request.getTargetGroup());
+        checkPermission("moveServers");
         groupAdminServer.moveServers(hostPorts, request.getTargetGroup());
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
@@ -187,13 +213,14 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
     public void moveTables(RpcController controller, MoveTablesRequest request,
         RpcCallback<MoveTablesResponse> done) {
       MoveTablesResponse.Builder builder = MoveTablesResponse.newBuilder();
+      Set<TableName> tables = new HashSet<>(request.getTableNameList().size());
+      for (TableProtos.TableName tableName : request.getTableNameList()) {
+        tables.add(ProtobufUtil.toTableName(tableName));
+      }
+      LOG.info(master.getClientIdAuditPrefix() + " move tables " + tables +" to rsgroup "
+          + request.getTargetGroup());
       try {
-        Set<TableName> tables = new HashSet<>(request.getTableNameList().size());
-        for (TableProtos.TableName tableName : request.getTableNameList()) {
-          tables.add(ProtobufUtil.toTableName(tableName));
-        }
-        LOG.info(master.getClientIdAuditPrefix() + " move tables " + tables +" to rsgroup "
-                + request.getTargetGroup());
+        checkPermission("moveTables");
         groupAdminServer.moveTables(tables, request.getTargetGroup());
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
@@ -207,6 +234,7 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       AddRSGroupResponse.Builder builder = AddRSGroupResponse.newBuilder();
       LOG.info(master.getClientIdAuditPrefix() + " add rsgroup " + request.getRSGroupName());
       try {
+        checkPermission("addRSGroup");
         groupAdminServer.addRSGroup(request.getRSGroupName());
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
@@ -221,6 +249,7 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
           RemoveRSGroupResponse.newBuilder();
       LOG.info(master.getClientIdAuditPrefix() + " remove rsgroup " + request.getRSGroupName());
       try {
+        checkPermission("removeRSGroup");
         groupAdminServer.removeRSGroup(request.getRSGroupName());
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
@@ -232,8 +261,10 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
     public void balanceRSGroup(RpcController controller,
         BalanceRSGroupRequest request, RpcCallback<BalanceRSGroupResponse> done) {
       BalanceRSGroupResponse.Builder builder = BalanceRSGroupResponse.newBuilder();
-      LOG.info(master.getClientIdAuditPrefix() + " balance rsgroup, group=" + request.getRSGroupName());
+      LOG.info(master.getClientIdAuditPrefix() + " balance rsgroup, group="
+          + request.getRSGroupName());
       try {
+        checkPermission("balanceRSGroup");
         builder.setBalanceRan(groupAdminServer.balanceRSGroup(request.getRSGroupName()));
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
@@ -248,6 +279,7 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       ListRSGroupInfosResponse.Builder builder = ListRSGroupInfosResponse.newBuilder();
       LOG.info(master.getClientIdAuditPrefix() + " list rsgroup");
       try {
+        checkPermission("listRSGroup");
         for (RSGroupInfo RSGroupInfo : groupAdminServer.listRSGroups()) {
           builder.addRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
         }
@@ -261,10 +293,12 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
     public void getRSGroupInfoOfServer(RpcController controller,
         GetRSGroupInfoOfServerRequest request, RpcCallback<GetRSGroupInfoOfServerResponse> done) {
       GetRSGroupInfoOfServerResponse.Builder builder = GetRSGroupInfoOfServerResponse.newBuilder();
+      Address hp = Address.fromParts(request.getServer().getHostName(),
+          request.getServer().getPort());
+      LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, server="
+          + hp);
       try {
-        Address hp = Address.fromParts(request.getServer().getHostName(),
-            request.getServer().getPort());
-        LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, server=" + hp);
+        checkPermission("getRSGroupInfoOfServer");
         RSGroupInfo RSGroupInfo = groupAdminServer.getRSGroupOfServer(hp);
         if (RSGroupInfo != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
@@ -279,17 +313,18 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
     public void moveServersAndTables(RpcController controller,
         MoveServersAndTablesRequest request, RpcCallback<MoveServersAndTablesResponse> done) {
       MoveServersAndTablesResponse.Builder builder = MoveServersAndTablesResponse.newBuilder();
+      Set<Address> hostPorts = Sets.newHashSet();
+      for (HBaseProtos.ServerName el : request.getServersList()) {
+        hostPorts.add(Address.fromParts(el.getHostName(), el.getPort()));
+      }
+      Set<TableName> tables = new HashSet<>(request.getTableNameList().size());
+      for (TableProtos.TableName tableName : request.getTableNameList()) {
+        tables.add(ProtobufUtil.toTableName(tableName));
+      }
+      LOG.info(master.getClientIdAuditPrefix() + " move servers " + hostPorts
+          + " and tables " + tables + " to rsgroup" + request.getTargetGroup());
       try {
-        Set<Address> hostPorts = Sets.newHashSet();
-        for (HBaseProtos.ServerName el : request.getServersList()) {
-          hostPorts.add(Address.fromParts(el.getHostName(), el.getPort()));
-        }
-        Set<TableName> tables = new HashSet<>(request.getTableNameList().size());
-        for (TableProtos.TableName tableName : request.getTableNameList()) {
-          tables.add(ProtobufUtil.toTableName(tableName));
-        }
-        LOG.info(master.getClientIdAuditPrefix() + " move servers " + hostPorts
-                + " and tables " + tables + " to rsgroup" + request.getTargetGroup());
+        checkPermission("moveServersAndTables");
         groupAdminServer.moveServersAndTables(hostPorts, tables, request.getTargetGroup());
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
@@ -303,13 +338,14 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
         RpcCallback<RemoveServersResponse> done) {
       RemoveServersResponse.Builder builder =
           RemoveServersResponse.newBuilder();
+      Set<Address> servers = Sets.newHashSet();
+      for (HBaseProtos.ServerName el : request.getServersList()) {
+        servers.add(Address.fromParts(el.getHostName(), el.getPort()));
+      }
+      LOG.info(master.getClientIdAuditPrefix()
+          + " remove decommissioned servers from rsgroup: " + servers);
       try {
-        Set<Address> servers = Sets.newHashSet();
-        for (HBaseProtos.ServerName el : request.getServersList()) {
-          servers.add(Address.fromParts(el.getHostName(), el.getPort()));
-        }
-        LOG.info(master.getClientIdAuditPrefix()
-            + " remove decommissioned servers from rsgroup: " + servers);
+        checkPermission("removeServers");
         groupAdminServer.removeServers(servers);
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
@@ -395,5 +431,21 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
     groupAdminServer.removeServers(clearedServer);
   }
 
-  /////////////////////////////////////////////////////////////////////////////
+  public void checkPermission(String request) throws IOException {
+    accessChecker.requirePermission(getActiveUser(), request, Action.ADMIN);
+  }
+
+  /**
+   * Returns the active user to which authorization checks should be applied.
+   * If we are in the context of an RPC call, the remote user is used,
+   * otherwise the currently logged in user is used.
+   */
+  private User getActiveUser() throws IOException {
+    // for non-rpc handling, fallback to system user
+    Optional<User> optionalUser = RpcServer.getRequestUser();
+    if (optionalUser.isPresent()) {
+      return optionalUser.get();
+    }
+    return userProvider.getCurrent();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fa3dcd9/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
new file mode 100644
index 0000000..1018dfa
--- /dev/null
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
@@ -0,0 +1,329 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hbase.rsgroup;
+
+import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.AccessControlLists;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.SecureTestUtil;
+import org.apache.hadoop.hbase.security.access.TableAuthManager;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Performs authorization checks for rsgroup operations, according to different
+ * levels of authorized users.
+ */
+@Category({SecurityTests.class})
+public class TestRSGroupsWithACL extends SecureTestUtil{
+  private static final Logger LOG = LoggerFactory.getLogger(TestRSGroupsWithACL.class);
+  private static TableName TEST_TABLE = TableName.valueOf("testtable1");
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf;
+
+  private static Connection systemUserConnection;
+  // user with all permissions
+  private static User SUPERUSER;
+  // user granted with all global permission
+  private static User USER_ADMIN;
+  // user with rw permissions on column family.
+  private static User USER_RW;
+  // user with read-only permissions
+  private static User USER_RO;
+  // user is table owner. will have all permissions on table
+  private static User USER_OWNER;
+  // user with create table permissions alone
+  private static User USER_CREATE;
+  // user with no permissions
+  private static User USER_NONE;
+
+  private static final String GROUP_ADMIN = "group_admin";
+  private static final String GROUP_CREATE = "group_create";
+  private static final String GROUP_READ = "group_read";
+  private static final String GROUP_WRITE = "group_write";
+
+  private static User USER_GROUP_ADMIN;
+  private static User USER_GROUP_CREATE;
+  private static User USER_GROUP_READ;
+  private static User USER_GROUP_WRITE;
+
+  private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
+
+  private static RSGroupAdminEndpoint rsGroupAdminEndpoint;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    // setup configuration
+    conf = TEST_UTIL.getConfiguration();
+    conf.set(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
+        RSGroupBasedLoadBalancer.class.getName());
+    // Enable security
+    enableSecurity(conf);
+    // Verify enableSecurity sets up what we require
+    verifyConfiguration(conf);
+    // Enable rsgroup
+    configureRSGroupAdminEndpoint(conf);
+
+    TEST_UTIL.startMiniCluster();
+    rsGroupAdminEndpoint = (RSGroupAdminEndpoint) TEST_UTIL.getMiniHBaseCluster().getMaster().
+        getMasterCoprocessorHost().findCoprocessor(RSGroupAdminEndpoint.class.getName());
+    // Wait for the ACL table to become available
+    TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);
+
+    // create a set of test users
+    SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
+    USER_ADMIN = User.createUserForTesting(conf, "admin2", new String[0]);
+    USER_RW = User.createUserForTesting(conf, "rwuser", new String[0]);
+    USER_RO = User.createUserForTesting(conf, "rouser", new String[0]);
+    USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
+    USER_CREATE = User.createUserForTesting(conf, "tbl_create", new String[0]);
+    USER_NONE = User.createUserForTesting(conf, "nouser", new String[0]);
+
+    USER_GROUP_ADMIN =
+        User.createUserForTesting(conf, "user_group_admin", new String[] { GROUP_ADMIN });
+    USER_GROUP_CREATE =
+        User.createUserForTesting(conf, "user_group_create", new String[] { GROUP_CREATE });
+    USER_GROUP_READ =
+        User.createUserForTesting(conf, "user_group_read", new String[] { GROUP_READ });
+    USER_GROUP_WRITE =
+        User.createUserForTesting(conf, "user_group_write", new String[] { GROUP_WRITE });
+
+    systemUserConnection = TEST_UTIL.getConnection();
+    setUpTableAndUserPermissions();
+  }
+
+  private static void setUpTableAndUserPermissions() throws Exception {
+    TableDescriptorBuilder tableBuilder = TableDescriptorBuilder.newBuilder(TEST_TABLE);
+    ColumnFamilyDescriptorBuilder cfd = ColumnFamilyDescriptorBuilder.newBuilder(TEST_FAMILY);
+    cfd.setMaxVersions(100);
+    tableBuilder.addColumnFamily(cfd.build());
+    tableBuilder.setValue(TableDescriptorBuilder.OWNER, USER_OWNER.getShortName());
+    createTable(TEST_UTIL, tableBuilder.build(),
+        new byte[][] { Bytes.toBytes("s") });
+
+    // Set up initial grants
+    grantGlobal(TEST_UTIL, USER_ADMIN.getShortName(),
+        Permission.Action.ADMIN,
+        Permission.Action.CREATE,
+        Permission.Action.READ,
+        Permission.Action.WRITE);
+
+    grantOnTable(TEST_UTIL, USER_RW.getShortName(),
+        TEST_TABLE, TEST_FAMILY, null,
+        Permission.Action.READ,
+        Permission.Action.WRITE);
+
+    // USER_CREATE is USER_RW plus CREATE permissions
+    grantOnTable(TEST_UTIL, USER_CREATE.getShortName(),
+        TEST_TABLE, null, null,
+        Permission.Action.CREATE,
+        Permission.Action.READ,
+        Permission.Action.WRITE);
+
+    grantOnTable(TEST_UTIL, USER_RO.getShortName(),
+        TEST_TABLE, TEST_FAMILY, null,
+        Permission.Action.READ);
+
+    grantGlobal(TEST_UTIL, toGroupEntry(GROUP_ADMIN), Permission.Action.ADMIN);
+    grantGlobal(TEST_UTIL, toGroupEntry(GROUP_CREATE), Permission.Action.CREATE);
+    grantGlobal(TEST_UTIL, toGroupEntry(GROUP_READ), Permission.Action.READ);
+    grantGlobal(TEST_UTIL, toGroupEntry(GROUP_WRITE), Permission.Action.WRITE);
+
+    assertEquals(5, AccessControlLists.getTablePermissions(conf, TEST_TABLE).size());
+    try {
+      assertEquals(5, AccessControlClient.getUserPermissions(systemUserConnection,
+          TEST_TABLE.toString()).size());
+    } catch (Throwable e) {
+      LOG.error("error during call of AccessControlClient.getUserPermissions. ", e);
+    }
+  }
+
+  private static void cleanUp() throws Exception {
+    // Clean the _acl_ table
+    try {
+      deleteTable(TEST_UTIL, TEST_TABLE);
+    } catch (TableNotFoundException ex) {
+      // Test deleted the table, no problem
+      LOG.info("Test deleted table " + TEST_TABLE);
+    }
+    // Verify all table/namespace permissions are erased
+    assertEquals(0, AccessControlLists.getTablePermissions(conf, TEST_TABLE).size());
+    assertEquals(0, AccessControlLists.getNamespacePermissions(conf,
+            TEST_TABLE.getNamespaceAsString()).size());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    cleanUp();
+    TEST_UTIL.shutdownMiniCluster();
+    int total = TableAuthManager.getTotalRefCount();
+    assertTrue("Unexpected reference count: " + total, total == 0);
+  }
+
+  private static void configureRSGroupAdminEndpoint(Configuration conf) {
+    String currentCoprocessors = conf.get(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
+    String coprocessors = RSGroupAdminEndpoint.class.getName();
+    if (currentCoprocessors != null) {
+      coprocessors += "," + currentCoprocessors;
+    }
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, coprocessors);
+    conf.set(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
+        RSGroupBasedLoadBalancer.class.getName());
+  }
+
+  @Test
+  public void testGetRSGroupInfo() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("getRSGroupInfo");
+      return null;
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+
+  @Test
+  public void testGetRSGroupInfoOfTable() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("getRSGroupInfoOfTable");
+      return null;
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+
+  @Test
+  public void testMoveServers() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("moveServers");
+      return null;
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+
+  @Test
+  public void testMoveTables() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("moveTables");
+      return null;
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+
+  @Test
+  public void testAddRSGroup() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("addRSGroup");
+      return null;
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+
+  @Test
+  public void testRemoveRSGroup() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("removeRSGroup");
+      return null;
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+
+  @Test
+  public void testBalanceRSGroup() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("balanceRSGroup");
+      return null;
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+
+  @Test
+  public void testListRSGroup() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("listRSGroup");
+      return null;
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+
+  @Test
+  public void testGetRSGroupInfoOfServer() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("getRSGroupInfoOfServer");
+      return null;
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+
+  @Test
+  public void testMoveServersAndTables() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("moveServersAndTables");
+      return null;
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fa3dcd9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 8f41e4f..907ca9b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessController;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
@@ -1810,7 +1811,7 @@ public class MasterRpcServices extends RSRpcServices
       // A coprocessor that implements AccessControlService can provide AUTHORIZATION and
       // CELL_AUTHORIZATION
       if (master.cpHost != null && hasAccessControlServiceCoprocessor(master.cpHost)) {
-        if (AccessController.isAuthorizationSupported(master.getConfiguration())) {
+        if (AccessChecker.isAuthorizationSupported(master.getConfiguration())) {
           capabilities.add(SecurityCapabilitiesResponse.Capability.AUTHORIZATION);
         }
         if (AccessController.isCellAuthorizationSupported(master.getConfiguration())) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fa3dcd9/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
new file mode 100644
index 0000000..d88e522
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
@@ -0,0 +1,332 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hbase.security.access;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public final class AccessChecker {
+  private static final Logger AUDITLOG =
+      LoggerFactory.getLogger("SecurityLogger." + AccessChecker.class.getName());
+  private TableAuthManager authManager;
+  /**
+   * if we are active, usually false, only true if "hbase.security.authorization"
+   * has been set to true in site configuration.see HBASE-19483.
+   */
+  private boolean authorizationEnabled;
+
+  public static boolean isAuthorizationSupported(Configuration conf) {
+    return conf.getBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, false);
+  }
+
+  /**
+   * Constructor with existing configuration
+   *
+   * @param conf Existing configuration to use
+   * @param zkw reference to the {@link ZKWatcher}
+   */
+  public AccessChecker(final Configuration conf, final ZKWatcher zkw)
+      throws RuntimeException {
+    // If zk is null or IOException while obtaining auth manager,
+    // throw RuntimeException so that the coprocessor is unloaded.
+    if (zkw != null) {
+      try {
+        this.authManager = TableAuthManager.getOrCreate(zkw, conf);
+      } catch (IOException ioe) {
+        throw new RuntimeException("Error obtaining AccessChecker", ioe);
+      }
+    } else {
+      throw new NullPointerException("Error obtaining AccessChecker, zk found null.");
+    }
+    authorizationEnabled = isAuthorizationSupported(conf);
+  }
+
+  public TableAuthManager getAuthManager() {
+    return authManager;
+  }
+
+  /**
+   * Authorizes that the current user has any of the given permissions to access the table.
+   *
+   * @param tableName   Table requested
+   * @param permissions Actions being requested
+   * @throws IOException           if obtaining the current user fails
+   * @throws AccessDeniedException if user has no authorization
+   */
+  public void requireAccess(User user, String request, TableName tableName,
+      Action... permissions) throws IOException {
+    AuthResult result = null;
+
+    for (Action permission : permissions) {
+      if (authManager.hasAccess(user, tableName, permission)) {
+        result = AuthResult.allow(request, "Table permission granted",
+            user, permission, tableName, null, null);
+        break;
+      } else {
+        // rest of the world
+        result = AuthResult.deny(request, "Insufficient permissions",
+            user, permission, tableName, null, null);
+      }
+    }
+    logResult(result);
+    if (authorizationEnabled && !result.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
+    }
+  }
+
+  /**
+   * Authorizes that the current user has global privileges for the given action.
+   *
+   * @param perm The action being requested
+   * @throws IOException           if obtaining the current user fails
+   * @throws AccessDeniedException if authorization is denied
+   */
+  public void requirePermission(User user, String request, Action perm)
+      throws IOException {
+    requireGlobalPermission(user, request, perm, null, null);
+  }
+
+  /**
+   * Checks that the user has the given global permission. The generated
+   * audit log message will contain context information for the operation
+   * being authorized, based on the given parameters.
+   *
+   * @param perm      Action being requested
+   * @param tableName Affected table name.
+   * @param familyMap Affected column families.
+   */
+  public void requireGlobalPermission(User user, String request,
+      Action perm, TableName tableName,
+      Map<byte[], ? extends Collection<byte[]>> familyMap)throws IOException {
+    AuthResult result;
+    if (authManager.authorize(user, perm)) {
+      result = AuthResult.allow(request, "Global check allowed",
+          user, perm, tableName, familyMap);
+      result.getParams().setTableName(tableName).setFamilies(familyMap);
+      logResult(result);
+    } else {
+      result = AuthResult.deny(request, "Global check failed",
+          user, perm, tableName, familyMap);
+      result.getParams().setTableName(tableName).setFamilies(familyMap);
+      logResult(result);
+      if (authorizationEnabled) {
+        throw new AccessDeniedException(
+            "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
+                + "' (global, action=" + perm.toString() + ")");
+      }
+    }
+  }
+
+  /**
+   * Checks that the user has the given global permission. The generated
+   * audit log message will contain context information for the operation
+   * being authorized, based on the given parameters.
+   *
+   * @param perm      Action being requested
+   * @param namespace The given namespace
+   */
+  public void requireGlobalPermission(User user, String request, Action perm,
+      String namespace) throws IOException {
+    AuthResult authResult;
+    if (authManager.authorize(user, perm)) {
+      authResult = AuthResult.allow(request, "Global check allowed",
+          user, perm, null);
+      authResult.getParams().setNamespace(namespace);
+      logResult(authResult);
+    } else {
+      authResult = AuthResult.deny(request, "Global check failed",
+          user, perm, null);
+      authResult.getParams().setNamespace(namespace);
+      logResult(authResult);
+      if (authorizationEnabled) {
+        throw new AccessDeniedException(
+            "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
+                + "' (global, action=" + perm.toString() + ")");
+      }
+    }
+  }
+
+  /**
+   * Checks that the user has the given global or namespace permission.
+   *
+   * @param namespace  The given namespace
+   * @param permissions Actions being requested
+   */
+  public void requireNamespacePermission(User user, String request, String namespace,
+      Action... permissions) throws IOException {
+    AuthResult result = null;
+
+    for (Action permission : permissions) {
+      if (authManager.authorize(user, namespace, permission)) {
+        result =
+            AuthResult.allow(request, "Namespace permission granted",
+                user, permission, namespace);
+        break;
+      } else {
+        // rest of the world
+        result = AuthResult.deny(request, "Insufficient permissions",
+            user, permission, namespace);
+      }
+    }
+    logResult(result);
+    if (authorizationEnabled && !result.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
+    }
+  }
+
+  /**
+   * Checks that the user has the given global or namespace permission.
+   *
+   * @param namespace  The given namespace
+   * @param permissions Actions being requested
+   */
+  public void requireNamespacePermission(User user, String request, String namespace,
+      TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap,
+      Action... permissions) throws IOException {
+    AuthResult result = null;
+
+    for (Action permission : permissions) {
+      if (authManager.authorize(user, namespace, permission)) {
+        result =
+            AuthResult.allow(request, "Namespace permission granted",
+                user, permission, namespace);
+        result.getParams().setTableName(tableName).setFamilies(familyMap);
+        break;
+      } else {
+        // rest of the world
+        result = AuthResult.deny(request, "Insufficient permissions",
+            user, permission, namespace);
+        result.getParams().setTableName(tableName).setFamilies(familyMap);
+      }
+    }
+    logResult(result);
+    if (authorizationEnabled && !result.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
+    }
+  }
+
+  /**
+   * Authorizes that the current user has any of the given permissions for the
+   * given table, column family and column qualifier.
+   *
+   * @param tableName Table requested
+   * @param family    Column family requested
+   * @param qualifier Column qualifier requested
+   * @throws IOException           if obtaining the current user fails
+   * @throws AccessDeniedException if user has no authorization
+   */
+  public void requirePermission(User user, String request, TableName tableName, byte[] family,
+      byte[] qualifier, Action... permissions) throws IOException {
+    AuthResult result = null;
+
+    for (Action permission : permissions) {
+      if (authManager.authorize(user, tableName, family, qualifier, permission)) {
+        result = AuthResult.allow(request, "Table permission granted",
+            user, permission, tableName, family,
+                qualifier);
+        break;
+      } else {
+        // rest of the world
+        result = AuthResult.deny(request, "Insufficient permissions",
+                user, permission, tableName, family,
+                qualifier);
+      }
+    }
+    logResult(result);
+    if (authorizationEnabled && !result.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
+    }
+  }
+
+  /**
+   * Authorizes that the current user has any of the given permissions for the
+   * given table, column family and column qualifier.
+   *
+   * @param tableName Table requested
+   * @param family    Column family param
+   * @param qualifier Column qualifier param
+   * @throws IOException           if obtaining the current user fails
+   * @throws AccessDeniedException if user has no authorization
+   */
+  public void requireTablePermission(User user, String request,
+      TableName tableName,byte[] family, byte[] qualifier,
+      Action... permissions) throws IOException {
+    AuthResult result = null;
+
+    for (Action permission : permissions) {
+      if (authManager.authorize(user, tableName, null, null, permission)) {
+        result = AuthResult.allow(request, "Table permission granted",
+            user, permission, tableName, null, null);
+        result.getParams().setFamily(family).setQualifier(qualifier);
+        break;
+      } else {
+        // rest of the world
+        result = AuthResult.deny(request, "Insufficient permissions",
+                user, permission, tableName, family, qualifier);
+        result.getParams().setFamily(family).setQualifier(qualifier);
+      }
+    }
+    logResult(result);
+    if (authorizationEnabled && !result.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
+    }
+  }
+
+  public void checkLockPermissions(User user, String namespace,
+      TableName tableName, RegionInfo[] regionInfos, String reason)
+      throws IOException {
+    if (namespace != null && !namespace.isEmpty()) {
+      requireNamespacePermission(user, reason, namespace, Action.ADMIN, Action.CREATE);
+    } else if (tableName != null || (regionInfos != null && regionInfos.length > 0)) {
+      // So, either a table or regions op. If latter, check perms ons table.
+      TableName tn = tableName != null? tableName: regionInfos[0].getTable();
+      requireTablePermission(user, reason, tn, null, null,
+          Action.ADMIN, Action.CREATE);
+    } else {
+      throw new DoNotRetryIOException("Invalid lock level when requesting permissions.");
+    }
+  }
+
+  public static void logResult(AuthResult result) {
+    if (AUDITLOG.isTraceEnabled()) {
+      AUDITLOG.trace("Access " + (result.isAllowed() ? "allowed" : "denied") + " for user " + (
+          result.getUser() != null ?
+              result.getUser().getShortName() :
+              "UNKNOWN") + "; reason: " + result.getReason() + "; remote address: "
+          + RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("")
+          + "; request: " + result.getRequest() + "; context: " + result.toContextString());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fa3dcd9/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index eb20c38..b0f33bd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -490,7 +491,8 @@ public class AccessControlLists {
     return getPermissions(conf, tableName != null ? tableName.getName() : null, null);
   }
 
-  static ListMultimap<String, TablePermission> getNamespacePermissions(Configuration conf,
+  @VisibleForTesting
+  public static ListMultimap<String, TablePermission> getNamespacePermissions(Configuration conf,
       String namespace) throws IOException {
     return getPermissions(conf, Bytes.toBytes(toNamespaceEntry(namespace)), null);
   }