You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by me...@apache.org on 2019/09/02 07:54:06 UTC

[hbase] branch branch-2.2 updated: HBASE-22946 Fix TableNotFound when grant/revoke if AccessController is not loaded (#561)

This is an automated email from the ASF dual-hosted git repository.

meiyi pushed a commit to branch branch-2.2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.2 by this push:
     new e3eca79  HBASE-22946 Fix TableNotFound when grant/revoke if AccessController is not loaded (#561)
e3eca79 is described below

commit e3eca7956034ab97f31954c2bbbff56ab8358b8b
Author: meiyi <me...@xiaomi.com>
AuthorDate: Mon Sep 2 15:45:43 2019 +0800

    HBASE-22946 Fix TableNotFound when grant/revoke if AccessController is not loaded (#561)
---
 .../hadoop/hbase/master/MasterRpcServices.java     | 200 +++++++++++----------
 .../access/TestUnloadAccessController.java         | 112 ++++++++++++
 2 files changed, 217 insertions(+), 95 deletions(-)

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 2ff036c..05c7c7a 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
@@ -2545,27 +2545,29 @@ public class MasterRpcServices extends RSRpcServices
   public GrantResponse grant(RpcController controller, GrantRequest request)
       throws ServiceException {
     try {
-      final UserPermission perm =
-          ShadedAccessControlUtil.toUserPermission(request.getUserPermission());
-      boolean mergeExistingPermissions = request.getMergeExistingPermissions();
-      if (master.cpHost != null) {
+      master.checkInitialized();
+      if (master.cpHost != null && hasAccessControlServiceCoprocessor(master.cpHost)) {
+        final UserPermission perm =
+            ShadedAccessControlUtil.toUserPermission(request.getUserPermission());
+        boolean mergeExistingPermissions = request.getMergeExistingPermissions();
         master.cpHost.preGrant(perm, mergeExistingPermissions);
-      }
-      try (Table table = master.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME)) {
-        AccessControlLists.addUserPermission(getConfiguration(), perm, table,
-          mergeExistingPermissions);
-      }
-      if (master.cpHost != null) {
+        try (Table table = master.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME)) {
+          AccessControlLists.addUserPermission(getConfiguration(), perm, table,
+            mergeExistingPermissions);
+        }
         master.cpHost.postGrant(perm, mergeExistingPermissions);
+        User caller = RpcServer.getRequestUser().orElse(null);
+        if (AUDITLOG.isTraceEnabled()) {
+          // audit log should store permission changes in addition to auth results
+          String remoteAddress = RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("");
+          AUDITLOG.trace("User {} (remote address: {}) granted permission {}", caller,
+            remoteAddress, perm);
+        }
+        return GrantResponse.getDefaultInstance();
+      } else {
+        throw new DoNotRetryIOException(
+            new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
       }
-      User caller = RpcServer.getRequestUser().orElse(null);
-      if (AUDITLOG.isTraceEnabled()) {
-        // audit log should store permission changes in addition to auth results
-        String remoteAddress = RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("");
-        AUDITLOG.trace("User {} (remote address: {}) granted permission {}", caller, remoteAddress,
-                perm);
-      }
-      return GrantResponse.getDefaultInstance();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
@@ -2575,25 +2577,27 @@ public class MasterRpcServices extends RSRpcServices
   public RevokeResponse revoke(RpcController controller, RevokeRequest request)
       throws ServiceException {
     try {
-      final UserPermission userPermission =
-          ShadedAccessControlUtil.toUserPermission(request.getUserPermission());
-      if (master.cpHost != null) {
+      master.checkInitialized();
+      if (master.cpHost != null && hasAccessControlServiceCoprocessor(master.cpHost)) {
+        final UserPermission userPermission =
+            ShadedAccessControlUtil.toUserPermission(request.getUserPermission());
         master.cpHost.preRevoke(userPermission);
-      }
-      try (Table table = master.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME)) {
-        AccessControlLists.removeUserPermission(master.getConfiguration(), userPermission, table);
-      }
-      if (master.cpHost != null) {
+        try (Table table = master.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME)) {
+          AccessControlLists.removeUserPermission(master.getConfiguration(), userPermission, table);
+        }
         master.cpHost.postRevoke(userPermission);
+        User caller = RpcServer.getRequestUser().orElse(null);
+        if (AUDITLOG.isTraceEnabled()) {
+          // audit log should record all permission changes
+          String remoteAddress = RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("");
+          AUDITLOG.trace("User {} (remote address: {}) revoked permission {}", caller,
+            remoteAddress, userPermission);
+        }
+        return RevokeResponse.getDefaultInstance();
+      } else {
+        throw new DoNotRetryIOException(
+            new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
       }
-      User caller = RpcServer.getRequestUser().orElse(null);
-      if (AUDITLOG.isTraceEnabled()) {
-        // audit log should record all permission changes
-        String remoteAddress = RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("");
-        AUDITLOG.trace("User {} (remote address: {}) revoked permission {}", caller, remoteAddress,
-                userPermission);
-      }
-      return RevokeResponse.getDefaultInstance();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
@@ -2603,48 +2607,51 @@ public class MasterRpcServices extends RSRpcServices
   public GetUserPermissionsResponse getUserPermissions(RpcController controller,
       GetUserPermissionsRequest request) throws ServiceException {
     try {
-      final String userName = request.hasUserName() ? request.getUserName().toStringUtf8() : null;
-      String namespace =
-          request.hasNamespaceName() ? request.getNamespaceName().toStringUtf8() : null;
-      TableName table =
-          request.hasTableName() ? ProtobufUtil.toTableName(request.getTableName()) : null;
-      byte[] cf = request.hasColumnFamily() ? request.getColumnFamily().toByteArray() : null;
-      byte[] cq = request.hasColumnQualifier() ? request.getColumnQualifier().toByteArray() : null;
-      Type permissionType = request.hasType() ? request.getType() : null;
-      if (master.cpHost != null) {
+      master.checkInitialized();
+      if (master.cpHost != null && hasAccessControlServiceCoprocessor(master.cpHost)) {
+        final String userName = request.hasUserName() ? request.getUserName().toStringUtf8() : null;
+        String namespace =
+            request.hasNamespaceName() ? request.getNamespaceName().toStringUtf8() : null;
+        TableName table =
+            request.hasTableName() ? ProtobufUtil.toTableName(request.getTableName()) : null;
+        byte[] cf = request.hasColumnFamily() ? request.getColumnFamily().toByteArray() : null;
+        byte[] cq =
+            request.hasColumnQualifier() ? request.getColumnQualifier().toByteArray() : null;
+        Type permissionType = request.hasType() ? request.getType() : null;
         master.getMasterCoprocessorHost().preGetUserPermissions(userName, namespace, table, cf, cq);
-      }
 
-      List<UserPermission> perms = null;
-      if (permissionType == Type.Table) {
-        boolean filter = (cf != null || userName != null) ? true : false;
-        perms = AccessControlLists.getUserTablePermissions(master.getConfiguration(), table, cf, cq,
-          userName, filter);
-      } else if (permissionType == Type.Namespace) {
-        perms = AccessControlLists.getUserNamespacePermissions(master.getConfiguration(), namespace,
-          userName, userName != null ? true : false);
-      } else {
-        perms = AccessControlLists.getUserPermissions(master.getConfiguration(), null, null, null,
-          userName, userName != null ? true : false);
-        // Skip super users when filter user is specified
-        if (userName == null) {
-          // Adding superusers explicitly to the result set as AccessControlLists do not store
-          // them. Also using acl as table name to be inline with the results of global admin and
-          // will help in avoiding any leakage of information about being superusers.
-          for (String user : Superusers.getSuperUsers()) {
-            perms.add(new UserPermission(user,
-                Permission.newBuilder().withActions(Action.values()).build()));
+        List<UserPermission> perms = null;
+        if (permissionType == Type.Table) {
+          boolean filter = (cf != null || userName != null) ? true : false;
+          perms = AccessControlLists.getUserTablePermissions(master.getConfiguration(), table, cf,
+            cq, userName, filter);
+        } else if (permissionType == Type.Namespace) {
+          perms = AccessControlLists.getUserNamespacePermissions(master.getConfiguration(),
+            namespace, userName, userName != null ? true : false);
+        } else {
+          perms = AccessControlLists.getUserPermissions(master.getConfiguration(), null, null, null,
+            userName, userName != null ? true : false);
+          // Skip super users when filter user is specified
+          if (userName == null) {
+            // Adding superusers explicitly to the result set as AccessControlLists do not store
+            // them. Also using acl as table name to be inline with the results of global admin and
+            // will help in avoiding any leakage of information about being superusers.
+            for (String user : Superusers.getSuperUsers()) {
+              perms.add(new UserPermission(user,
+                  Permission.newBuilder().withActions(Action.values()).build()));
+            }
           }
         }
-      }
 
-      if (master.cpHost != null) {
         master.getMasterCoprocessorHost().postGetUserPermissions(userName, namespace, table, cf,
           cq);
+        AccessControlProtos.GetUserPermissionsResponse response =
+            ShadedAccessControlUtil.buildGetUserPermissionsResponse(perms);
+        return response;
+      } else {
+        throw new DoNotRetryIOException(
+            new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
       }
-      AccessControlProtos.GetUserPermissionsResponse response =
-          ShadedAccessControlUtil.buildGetUserPermissionsResponse(perms);
-      return response;
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
@@ -2654,38 +2661,41 @@ public class MasterRpcServices extends RSRpcServices
   public HasUserPermissionsResponse hasUserPermissions(RpcController controller,
       HasUserPermissionsRequest request) throws ServiceException {
     try {
-      User caller = RpcServer.getRequestUser().orElse(null);
-      String userName =
-          request.hasUserName() ? request.getUserName().toStringUtf8() : caller.getShortName();
-      List<Permission> permissions = new ArrayList<>();
-      for (int i = 0; i < request.getPermissionCount(); i++) {
-        permissions.add(ShadedAccessControlUtil.toPermission(request.getPermission(i)));
-      }
-      if (master.cpHost != null) {
+      master.checkInitialized();
+      if (master.cpHost != null && hasAccessControlServiceCoprocessor(master.cpHost)) {
+        User caller = RpcServer.getRequestUser().orElse(null);
+        String userName =
+            request.hasUserName() ? request.getUserName().toStringUtf8() : caller.getShortName();
+        List<Permission> permissions = new ArrayList<>();
+        for (int i = 0; i < request.getPermissionCount(); i++) {
+          permissions.add(ShadedAccessControlUtil.toPermission(request.getPermission(i)));
+        }
         master.getMasterCoprocessorHost().preHasUserPermissions(userName, permissions);
-      }
-      if (!caller.getShortName().equals(userName)) {
-        List<String> groups = AccessChecker.getUserGroups(userName);
-        caller = new InputUser(userName, groups.toArray(new String[groups.size()]));
-      }
-      List<Boolean> hasUserPermissions = new ArrayList<>();
-      if (accessChecker != null) {
-        for (Permission permission : permissions) {
-          boolean hasUserPermission =
-              accessChecker.hasUserPermission(caller, "hasUserPermissions", permission);
-          hasUserPermissions.add(hasUserPermission);
+
+        if (!caller.getShortName().equals(userName)) {
+          List<String> groups = AccessChecker.getUserGroups(userName);
+          caller = new InputUser(userName, groups.toArray(new String[groups.size()]));
         }
-      } else {
-        for (int i = 0; i < permissions.size(); i++) {
-          hasUserPermissions.add(true);
+        List<Boolean> hasUserPermissions = new ArrayList<>();
+        if (accessChecker != null) {
+          for (Permission permission : permissions) {
+            boolean hasUserPermission =
+                accessChecker.hasUserPermission(caller, "hasUserPermissions", permission);
+            hasUserPermissions.add(hasUserPermission);
+          }
+        } else {
+          for (int i = 0; i < permissions.size(); i++) {
+            hasUserPermissions.add(true);
+          }
         }
-      }
-      if (master.cpHost != null) {
         master.getMasterCoprocessorHost().postHasUserPermissions(userName, permissions);
+        HasUserPermissionsResponse.Builder builder =
+            HasUserPermissionsResponse.newBuilder().addAllHasUserPermission(hasUserPermissions);
+        return builder.build();
+      } else {
+        throw new DoNotRetryIOException(
+            new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
       }
-      HasUserPermissionsResponse.Builder builder =
-          HasUserPermissionsResponse.newBuilder().addAllHasUserPermission(hasUserPermissions);
-      return builder.build();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestUnloadAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestUnloadAccessController.java
new file mode 100644
index 0000000..2c9b3c1
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestUnloadAccessController.java
@@ -0,0 +1,112 @@
+/**
+ * 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 static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ SecurityTests.class, SmallTests.class })
+public class TestUnloadAccessController extends SecureTestUtil {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestUnloadAccessController.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static TableName TEST_TABLE = TableName.valueOf("TestUnloadAccessController");
+  private static Permission permission =
+      Permission.newBuilder(TEST_TABLE).withActions(Permission.Action.READ).build();
+  private static Admin admin;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    TEST_UTIL.startMiniCluster();
+    TEST_UTIL.waitUntilAllSystemRegionsAssigned();
+    admin = TEST_UTIL.getAdmin();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testGrant() {
+    try {
+      admin.grant(new UserPermission("user", permission), false);
+      fail("Expected UnsupportedOperationException but not found");
+    } catch (Throwable e) {
+      checkException(e);
+    }
+  }
+
+  @Test
+  public void testRevoke() {
+    try {
+      admin.revoke(new UserPermission("user", permission));
+      fail("Expected UnsupportedOperationException but not found");
+    } catch (Throwable e) {
+      e.printStackTrace();
+      checkException(e);
+    }
+  }
+
+  @Test
+  public void testGetUserPermissions() {
+    try {
+      admin.getUserPermissions(GetUserPermissionsRequest.newBuilder().build());
+      fail("Expected UnsupportedOperationException but not found");
+    } catch (Throwable e) {
+      checkException(e);
+    }
+  }
+
+  @Test
+  public void testHasUserPermission() {
+    try {
+      List<Permission> permissionList = new ArrayList<>();
+      permissionList.add(permission);
+      admin.hasUserPermissions(permissionList);
+      fail("Expected UnsupportedOperationException but not found");
+    } catch (Throwable e) {
+      checkException(e);
+    }
+  }
+
+  private void checkException(Throwable e) {
+    if (e instanceof DoNotRetryIOException
+        && e.getMessage().contains(UnsupportedOperationException.class.getName())) {
+      return;
+    }
+    fail("Expected UnsupportedOperationException but found " + e.getMessage());
+  }
+}