You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by mi...@apache.org on 2019/05/24 14:20:54 UTC

[phoenix] branch 4.x-HBase-1.5 updated: PHOENIX-5269 PhoenixAccessController should use AccessChecker instead of AccessControlClient for permission checks

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

mihir6692 pushed a commit to branch 4.x-HBase-1.5
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/4.x-HBase-1.5 by this push:
     new 3b92ec3  PHOENIX-5269 PhoenixAccessController should use AccessChecker instead of AccessControlClient for permission checks
3b92ec3 is described below

commit 3b92ec3f7fbbde03618a7cbea5e88b1a528ef6bc
Author: Kiran Kumar Maturi <ma...@gmail.com>
AuthorDate: Fri May 24 19:43:47 2019 +0530

    PHOENIX-5269 PhoenixAccessController should use AccessChecker instead of AccessControlClient for permission checks
---
 .../apache/phoenix/end2end/PermissionsCacheIT.java | 108 +++++++++++++++++++++
 .../coprocessor/PhoenixAccessController.java       |  89 +++++++++++++++--
 2 files changed, 189 insertions(+), 8 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PermissionsCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PermissionsCacheIT.java
new file mode 100644
index 0000000..ed36e63
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PermissionsCacheIT.java
@@ -0,0 +1,108 @@
+/*
+ * 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.phoenix.end2end;
+
+import static org.junit.Assert.assertTrue;
+
+import java.security.PrivilegedExceptionAction;
+import java.sql.Connection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.security.access.AccessControlLists;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.security.access.TablePermission;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.ListMultimap;
+
+public class PermissionsCacheIT extends BasePermissionsIT {
+
+    public PermissionsCacheIT() throws Exception {
+        super(true);
+    }
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        BasePermissionsIT.initCluster(true);
+    }
+    
+    @Test
+    public void testPermissionsCachedWithAccessChecker() throws Throwable {
+        if (!isNamespaceMapped) {
+            return;
+        }
+        final String schema = generateUniqueName();
+        final String tableName = generateUniqueName();
+        final String phoenixTableName = SchemaUtil.getTableName(schema, tableName);
+        try (Connection conn = getConnection()) {
+            grantPermissions(regularUser1.getShortName(), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES,
+                Action.READ, Action.EXEC);
+            grantPermissions(regularUser1.getShortName(), Collections.singleton("SYSTEM:SEQUENCE"),
+                Action.WRITE, Action.READ, Action.EXEC);
+            superUser1.runAs(new PrivilegedExceptionAction<Void>() {
+                @Override
+                public Void run() throws Exception {
+                    try {
+                        verifyAllowed(createSchema(schema), superUser1);
+                        grantPermissions(regularUser1.getShortName(), schema, Action.CREATE);
+                        grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), schema,
+                            Action.CREATE);
+                    } catch (Throwable e) {
+                        if (e instanceof Exception) {
+                            throw (Exception) e;
+                        } else {
+                            throw new Exception(e);
+                        }
+                    }
+                    return null;
+                }
+            });
+            verifyAllowed(createTable(phoenixTableName), regularUser1);
+            HBaseTestingUtility utility = getUtility();
+            Configuration conf = utility.getConfiguration();
+            ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
+            String aclZnodeParent = conf.get("zookeeper.znode.acl.parent", "acl");
+            String aclZNode = ZKUtil.joinZNode(zkw.baseZNode, aclZnodeParent);
+            String tableZNode = ZKUtil.joinZNode(aclZNode, "@" + schema);
+            byte[] data = ZKUtil.getData(zkw, tableZNode);
+            ListMultimap<String, TablePermission> userPermissions =
+                    AccessControlLists.readPermissions(data, conf);
+            assertTrue("User permissions not found in cache:",
+                userPermissions.containsKey(regularUser1.getName()));
+            List<TablePermission> tablePermissions = userPermissions.get(regularUser1.getName());
+            for (TablePermission tablePerm : tablePermissions) {
+                assertTrue("Table create permission don't exist", tablePerm.implies(Action.CREATE));
+            }
+        } catch (Exception e) {
+            System.out.println("Exception occurred: " + e);
+            throw e;
+        }
+        finally {
+            revokeAll();
+        }
+    }
+
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
index 59d3036..9db11b0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
@@ -27,11 +27,11 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 
-import com.google.protobuf.ByteString;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.coprocessor.BaseMasterAndRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -53,12 +55,15 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 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.AccessControlClient;
 import org.apache.hadoop.hbase.security.access.AuthResult;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.security.access.TableAuthManager;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -67,6 +72,7 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.MetaDataUtil;
 
+import com.google.protobuf.ByteString;
 import com.google.protobuf.RpcCallback;
 
 public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
@@ -74,7 +80,9 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
     private PhoenixMetaDataControllerEnvironment env;
     AtomicReference<ArrayList<BaseMasterAndRegionObserver>> accessControllers = new AtomicReference<>();
     private boolean accessCheckEnabled;
+    private boolean hbaseAccessControllerEnabled;
     private UserProvider userProvider;
+    private AccessChecker accessChecker;
     public static final Log LOG = LogFactory.getLog(PhoenixAccessController.class);
     private static final Log AUDITLOG =
             LogFactory.getLog("SecurityLogger."+PhoenixAccessController.class.getName());
@@ -88,6 +96,10 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
             for (BaseMasterAndRegionObserver cp : coprocessors) {
                 if (cp instanceof AccessControlService.Interface) {
                     oldAccessControllers.add(cp);
+                    if (cp.getClass().getName().equals(
+                        org.apache.hadoop.hbase.security.access.AccessController.class.getName())) {
+                        hbaseAccessControllerEnabled = true;
+                    }
                 }
             }
             accessControllers.set(oldAccessControllers);
@@ -116,6 +128,23 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
             throw new IllegalArgumentException(
                     "Not a valid environment, should be loaded by PhoenixMetaDataControllerEnvironment");
         }
+        CompoundConfiguration compoundConf = new CompoundConfiguration();
+        compoundConf.add(env.getConfiguration());
+        ZooKeeperWatcher zk = null;
+        if (env instanceof MasterCoprocessorEnvironment) {
+            // if running on HMaster
+            MasterCoprocessorEnvironment mEnv = (MasterCoprocessorEnvironment) env;
+            zk = mEnv.getMasterServices().getZooKeeper();
+          } else if (env instanceof RegionServerCoprocessorEnvironment) {
+            RegionServerCoprocessorEnvironment rsEnv = (RegionServerCoprocessorEnvironment) env;
+            zk = rsEnv.getRegionServerServices().getZooKeeper();
+          } else if (env instanceof RegionCoprocessorEnvironment) {
+            // if running at region
+            RegionCoprocessorEnvironment regionEnv = (RegionCoprocessorEnvironment) env;
+            compoundConf.addStringMap(regionEnv.getRegion().getTableDesc().getConfiguration());
+            zk = regionEnv.getRegionServerServices().getZooKeeper();
+        }
+        accessChecker = new AccessChecker(env.getConfiguration(), zk);
         // set the user-provider.
         this.userProvider = UserProvider.instantiate(env.getConfiguration());
         // init superusers and add the server principal (if using security)
@@ -124,7 +153,11 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
     }
 
     @Override
-    public void stop(CoprocessorEnvironment env) throws IOException {}
+    public void stop(CoprocessorEnvironment env) throws IOException {
+        if(accessChecker.getAuthManager() != null) {
+            TableAuthManager.release(accessChecker.getAuthManager());
+        }
+    }
 
     @Override
     public void preCreateTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
@@ -398,9 +431,10 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
      * @throws IOException
      */
     private List<UserPermission> getUserPermissions(final TableName tableName) throws IOException {
-        return User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
-            @Override
-            public List<UserPermission> run() throws Exception {
+        List<UserPermission> userPermissions =
+                User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
+                    @Override
+                    public List<UserPermission> run() throws Exception {
                 final List<UserPermission> userPermissions = new ArrayList<UserPermission>();
                 try (Connection connection = ConnectionFactory.createConnection(env.getConfiguration())) {
                     // Merge permissions from all accessController coprocessors loaded in memory
@@ -410,8 +444,6 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
                             userPermissions.addAll(AccessControlClient.getUserPermissions(connection, tableName.getNameAsString()));
                             userPermissions.addAll(AccessControlClient.getUserPermissions(
                                     connection, AuthUtil.toGroupEntry(tableName.getNamespaceAsString())));
-                        } else {
-                            getUserPermsFromUserDefinedAccessController(userPermissions, connection, (AccessControlService.Interface) service);
                         }
                     }
                 } catch (Throwable e) {
@@ -424,6 +456,35 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
                 }
                 return userPermissions;
             }
+        });
+        getUserDefinedPermissions(tableName, userPermissions);
+        return userPermissions;
+    }
+
+    private void getUserDefinedPermissions(final TableName tableName, final List<UserPermission> userPermissions) throws IOException {
+        User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
+            @Override
+            public List<UserPermission> run() throws Exception {
+                final List<UserPermission> userPermissions = new ArrayList<UserPermission>();
+                try (Connection connection =
+                        ConnectionFactory.createConnection(env.getConfiguration())) {
+                    for (BaseMasterAndRegionObserver service : getAccessControllers()) {
+                         if (service.getClass().getName().equals(org.apache.hadoop.hbase.security.access.AccessController.class.getName())) {
+                            continue;
+                         } else {
+                             getUserPermsFromUserDefinedAccessController(userPermissions, connection, (AccessControlService.Interface) service);
+                         }
+                    }
+                } catch (Throwable e) {
+                    if (e instanceof Exception) {
+                        throw (Exception) e;
+                    } else if (e instanceof Error) {
+                        throw (Error) e;
+                    }
+                    throw new Exception(e);
+                }
+                return userPermissions;
+            }
 
             private void getUserPermsFromUserDefinedAccessController(final List<UserPermission> userPermissions, Connection connection, AccessControlService.Interface service) {
 
@@ -477,8 +538,12 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
         User user = getActiveUser();
         AuthResult result = null;
         List<Action> requiredAccess = new ArrayList<Action>();
+        List<UserPermission> userPermissions = new ArrayList<>();
+        if(permissions.length > 0) {
+           getUserDefinedPermissions(tableName, userPermissions);
+        }
         for (Action permission : permissions) {
-             if (hasAccess(getUserPermissions(tableName), tableName, permission, user)) {
+             if (hasAccess(userPermissions, tableName, permission, user)) {
                 result = AuthResult.allow(request, "Table permission granted", user, permission, tableName, null, null);
             } else {
                 result = AuthResult.deny(request, "Insufficient permissions", user, permission, tableName, null, null);
@@ -506,6 +571,10 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
             return true;
         }
         if (perms != null) {
+            if (hbaseAccessControllerEnabled
+                    && accessChecker.getAuthManager().userHasAccess(user, table, action)) {
+                return true;
+            }
             List<UserPermission> permissionsForUser = getPermissionForUser(perms, user.getShortName().getBytes());
             if (permissionsForUser != null) {
                 for (UserPermission permissionForUser : permissionsForUser) {
@@ -516,6 +585,10 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
             if (groupNames != null) {
               for (String group : groupNames) {
                 List<UserPermission> groupPerms = getPermissionForUser(perms,(AuthUtil.toGroupEntry(group)).getBytes());
+                if (hbaseAccessControllerEnabled && accessChecker.getAuthManager()
+                        .groupHasAccess(group, table, action)) {
+                    return true;
+                }
                 if (groupPerms != null) for (UserPermission permissionForUser : groupPerms) {
                     if (permissionForUser.implies(action)) { return true; }
                 }