You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@phoenix.apache.org by karanmehta93 <gi...@git.apache.org> on 2017/11/17 01:37:27 UTC

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

GitHub user karanmehta93 opened a pull request:

    https://github.com/apache/phoenix/pull/283

    PHOENIX-672 Add GRANT and REVOKE commands using HBase AccessController

    Initial patch
    Added support for Grant Revoke SQL statements for HBase ACL Permissions
    Refactored the tests, BasePermissionsIT is the base test for Permission management tests and provides helper methods. 
    Index tables will be automatically granted permissions as base table when created
    
    Ongoing
    Removing console debug messages, adding log lines, making patch cleaner
    End to end testing with HBase and sqlline.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/karanmehta93/phoenix PHOENIX-672

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/phoenix/pull/283.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #283
    
----
commit 16138fc5a4846b6483d2ad2a2eeca7d01f3f1727
Author: Karan Mehta <ka...@gmail.com>
Date:   2017-11-17T01:29:33Z

    PHOENIX-672 Add GRANT and REVOKE commands using HBase AccessController

----


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r153582196
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java ---
    @@ -0,0 +1,635 @@
    +/*
    + * 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 com.google.common.base.Throwables;
    +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.HBaseTestingUtility;
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.security.AccessDeniedException;
    +import org.apache.hadoop.hbase.security.User;
    +import org.apache.hadoop.hbase.security.access.AccessControlClient;
    +import org.apache.hadoop.hbase.security.access.Permission;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.query.BaseTest;
    +import org.apache.phoenix.query.QueryConstants;
    +import org.apache.phoenix.query.QueryServices;
    +import org.junit.After;
    +import org.junit.BeforeClass;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +
    +import java.io.IOException;
    +import java.lang.reflect.UndeclaredThrowableException;
    +import java.security.PrivilegedExceptionAction;
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Statement;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertNotNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.fail;
    +
    +@RunWith(Parameterized.class)
    +public class BasePermissionsIT extends BaseTest {
    +
    +    private static final Log LOG = LogFactory.getLog(BasePermissionsIT.class);
    +
    +    static String SUPERUSER;
    +
    +    static HBaseTestingUtility testUtil;
    +    static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
    +            "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION"));
    +
    +    static final Set<String> PHOENIX_SYSTEM_TABLES_IDENTIFIERS = new HashSet<>(Arrays.asList(
    +            "SYSTEM.\"CATALOG\"", "SYSTEM.\"SEQUENCE\"", "SYSTEM.\"STATS\"", "SYSTEM.\"FUNCTION\""));
    +
    +    static final String SYSTEM_SEQUENCE_IDENTIFIER =
    +            QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\"" + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE+ "\"";
    +
    +    static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
    +            "SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION"));
    +
    +    // Create Multiple users so that we can use Hadoop UGI to run tasks as various users
    +    // Permissions can be granted or revoke by superusers and admins only
    +    // DON'T USE HADOOP UserGroupInformation class to create testing users since HBase misses some of its functionality
    +    // Instead use org.apache.hadoop.hbase.security.User class for testing purposes.
    +
    +    // Super User has all the access
    +    User superUser1 = null;
    +    User superUser2 = null;
    +
    +    // Regular users are granted and revoked permissions as needed
    +    User regularUser1 = null;
    +    User regularUser2 = null;
    +    User regularUser3 = null;
    +    User regularUser4 = null;
    +
    +    // Group User is equivalent of regular user but inside a group
    +    // Permissions can be granted to group should affect this user
    +    static final String GROUP_SYSTEM_ACCESS = "group_system_access";
    +    User groupUser = null;
    +
    +    // Unpriviledged User doesn't have any access and is denied for every action
    +    User unprivilegedUser = null;
    +
    +    static final int NUM_RECORDS = 5;
    +
    +    boolean isNamespaceMapped;
    +
    +    public BasePermissionsIT(final boolean isNamespaceMapped) throws Exception {
    +        this.isNamespaceMapped = isNamespaceMapped;
    +    }
    +
    +    @BeforeClass
    +    public static void doSetup() throws Exception {
    +        SUPERUSER = System.getProperty("user.name");
    +    }
    +
    +    void startNewMiniCluster() throws Exception {
    +        startNewMiniCluster(new Configuration());
    +    }
    +    
    +    void startNewMiniCluster(Configuration overrideConf) throws Exception{
    +        if (null != testUtil) {
    +            testUtil.shutdownMiniCluster();
    +            testUtil = null;
    +        }
    +
    +        testUtil = new HBaseTestingUtility();
    +
    +        Configuration config = testUtil.getConfiguration();
    +        enablePhoenixHBaseAuthorization(config);
    +        configureNamespacesOnServer(config);
    +        configureRandomHMasterPort(config);
    +        if (overrideConf != null) {
    +            config.addResource(overrideConf);
    +        }
    +
    +        testUtil.startMiniCluster(1);
    +        initializeUsers(testUtil.getConfiguration());
    +    }
    +
    +    private void initializeUsers(Configuration configuration) {
    +
    +        superUser1 = User.createUserForTesting(configuration, SUPERUSER, new String[0]);
    +        superUser2 = User.createUserForTesting(configuration, "superUser2", new String[0]);
    +
    +        regularUser1 = User.createUserForTesting(configuration, "regularUser1", new String[0]);
    +        regularUser2 = User.createUserForTesting(configuration, "regularUser2", new String[0]);
    +        regularUser3 = User.createUserForTesting(configuration, "regularUser3", new String[0]);
    +        regularUser4 = User.createUserForTesting(configuration, "regularUser4", new String[0]);
    +
    +        groupUser = User.createUserForTesting(testUtil.getConfiguration(), "groupUser", new String[] {GROUP_SYSTEM_ACCESS});
    +
    +        unprivilegedUser = User.createUserForTesting(configuration, "unprivilegedUser", new String[0]);
    +    }
    +
    +    private void configureRandomHMasterPort(Configuration config) {
    +        // Avoid multiple clusters trying to bind the master's info port (16010)
    +        config.setInt(HConstants.MASTER_INFO_PORT, -1);
    +    }
    +
    +    void enablePhoenixHBaseAuthorization(Configuration config) {
    +        config.set("hbase.superuser", SUPERUSER + "," + "superUser2");
    +        config.set("hbase.security.authorization", Boolean.TRUE.toString());
    +        config.set("hbase.security.exec.permission.checks", Boolean.TRUE.toString());
    +        config.set("hbase.coprocessor.master.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +        config.set("hbase.coprocessor.region.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +        config.set("hbase.coprocessor.regionserver.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +
    +        config.set(QueryServices.PHOENIX_ACLS_ENABLED,"true");
    +
    +        config.set("hbase.regionserver.wal.codec", "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec");
    +    }
    +
    +    void configureNamespacesOnServer(Configuration conf) {
    +        conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
    +    }
    +
    +    @Parameterized.Parameters(name = "isNamespaceMapped={0}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean> data() {
    +        return Arrays.asList(false, true);
    +    }
    +
    +    @After
    +    public void cleanup() throws Exception {
    +        if (testUtil != null) {
    +            testUtil.shutdownMiniCluster();
    +            testUtil = null;
    +        }
    +    }
    +
    +    public static HBaseTestingUtility getUtility(){
    +        return testUtil;
    +    }
    +
    --- End diff --
    
    As discussed yesterday, I am still keeping the old tests to use HBase API directly and my tests will use Phoenix GRANT/REVOKE. The old tests verify if a connection can be created and used with minimal permissions and it is good to have them.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152662904
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    --- End diff --
    
    Theoretically yes, However since the operation is not atomic, it may result into weird case where they are differing.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152645385
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    --- End diff --
    
    Sure. Will add. 


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152643463
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java ---
    @@ -229,17 +227,12 @@ public void handleRequireAccessOnDependentTable(String request, String userName,
                         + dependentTable);
                 return;
             }
    -        if (isAutomaticGrantEnabled) {
    --- End diff --
    
    I didn't quite clearly understand the difference between automatic grant and strict mode. Could you please elaborate @ankitsinghal?


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152691915
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java ---
    @@ -0,0 +1,635 @@
    +/*
    + * 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 com.google.common.base.Throwables;
    +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.HBaseTestingUtility;
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.security.AccessDeniedException;
    +import org.apache.hadoop.hbase.security.User;
    +import org.apache.hadoop.hbase.security.access.AccessControlClient;
    +import org.apache.hadoop.hbase.security.access.Permission;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.query.BaseTest;
    +import org.apache.phoenix.query.QueryConstants;
    +import org.apache.phoenix.query.QueryServices;
    +import org.junit.After;
    +import org.junit.BeforeClass;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +
    +import java.io.IOException;
    +import java.lang.reflect.UndeclaredThrowableException;
    +import java.security.PrivilegedExceptionAction;
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Statement;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertNotNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.fail;
    +
    +@RunWith(Parameterized.class)
    +public class BasePermissionsIT extends BaseTest {
    +
    +    private static final Log LOG = LogFactory.getLog(BasePermissionsIT.class);
    +
    +    static String SUPERUSER;
    +
    +    static HBaseTestingUtility testUtil;
    +    static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
    +            "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION"));
    +
    +    static final Set<String> PHOENIX_SYSTEM_TABLES_IDENTIFIERS = new HashSet<>(Arrays.asList(
    +            "SYSTEM.\"CATALOG\"", "SYSTEM.\"SEQUENCE\"", "SYSTEM.\"STATS\"", "SYSTEM.\"FUNCTION\""));
    +
    +    static final String SYSTEM_SEQUENCE_IDENTIFIER =
    +            QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\"" + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE+ "\"";
    +
    +    static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
    +            "SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION"));
    +
    +    // Create Multiple users so that we can use Hadoop UGI to run tasks as various users
    +    // Permissions can be granted or revoke by superusers and admins only
    +    // DON'T USE HADOOP UserGroupInformation class to create testing users since HBase misses some of its functionality
    +    // Instead use org.apache.hadoop.hbase.security.User class for testing purposes.
    +
    +    // Super User has all the access
    +    User superUser1 = null;
    +    User superUser2 = null;
    +
    +    // Regular users are granted and revoked permissions as needed
    +    User regularUser1 = null;
    +    User regularUser2 = null;
    +    User regularUser3 = null;
    +    User regularUser4 = null;
    +
    +    // Group User is equivalent of regular user but inside a group
    +    // Permissions can be granted to group should affect this user
    +    static final String GROUP_SYSTEM_ACCESS = "group_system_access";
    +    User groupUser = null;
    +
    +    // Unpriviledged User doesn't have any access and is denied for every action
    +    User unprivilegedUser = null;
    +
    +    static final int NUM_RECORDS = 5;
    +
    +    boolean isNamespaceMapped;
    +
    +    public BasePermissionsIT(final boolean isNamespaceMapped) throws Exception {
    +        this.isNamespaceMapped = isNamespaceMapped;
    +    }
    +
    +    @BeforeClass
    +    public static void doSetup() throws Exception {
    +        SUPERUSER = System.getProperty("user.name");
    +    }
    +
    +    void startNewMiniCluster() throws Exception {
    +        startNewMiniCluster(new Configuration());
    +    }
    +    
    +    void startNewMiniCluster(Configuration overrideConf) throws Exception{
    +        if (null != testUtil) {
    +            testUtil.shutdownMiniCluster();
    +            testUtil = null;
    +        }
    +
    +        testUtil = new HBaseTestingUtility();
    +
    +        Configuration config = testUtil.getConfiguration();
    +        enablePhoenixHBaseAuthorization(config);
    +        configureNamespacesOnServer(config);
    +        configureRandomHMasterPort(config);
    +        if (overrideConf != null) {
    +            config.addResource(overrideConf);
    +        }
    +
    +        testUtil.startMiniCluster(1);
    +        initializeUsers(testUtil.getConfiguration());
    +    }
    +
    +    private void initializeUsers(Configuration configuration) {
    +
    +        superUser1 = User.createUserForTesting(configuration, SUPERUSER, new String[0]);
    +        superUser2 = User.createUserForTesting(configuration, "superUser2", new String[0]);
    +
    +        regularUser1 = User.createUserForTesting(configuration, "regularUser1", new String[0]);
    +        regularUser2 = User.createUserForTesting(configuration, "regularUser2", new String[0]);
    +        regularUser3 = User.createUserForTesting(configuration, "regularUser3", new String[0]);
    +        regularUser4 = User.createUserForTesting(configuration, "regularUser4", new String[0]);
    +
    +        groupUser = User.createUserForTesting(testUtil.getConfiguration(), "groupUser", new String[] {GROUP_SYSTEM_ACCESS});
    +
    +        unprivilegedUser = User.createUserForTesting(configuration, "unprivilegedUser", new String[0]);
    +    }
    +
    +    private void configureRandomHMasterPort(Configuration config) {
    +        // Avoid multiple clusters trying to bind the master's info port (16010)
    +        config.setInt(HConstants.MASTER_INFO_PORT, -1);
    +    }
    +
    +    void enablePhoenixHBaseAuthorization(Configuration config) {
    +        config.set("hbase.superuser", SUPERUSER + "," + "superUser2");
    +        config.set("hbase.security.authorization", Boolean.TRUE.toString());
    +        config.set("hbase.security.exec.permission.checks", Boolean.TRUE.toString());
    +        config.set("hbase.coprocessor.master.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +        config.set("hbase.coprocessor.region.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +        config.set("hbase.coprocessor.regionserver.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +
    +        config.set(QueryServices.PHOENIX_ACLS_ENABLED,"true");
    +
    +        config.set("hbase.regionserver.wal.codec", "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec");
    +    }
    +
    +    void configureNamespacesOnServer(Configuration conf) {
    +        conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
    +    }
    +
    +    @Parameterized.Parameters(name = "isNamespaceMapped={0}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean> data() {
    +        return Arrays.asList(false, true);
    +    }
    +
    +    @After
    +    public void cleanup() throws Exception {
    +        if (testUtil != null) {
    +            testUtil.shutdownMiniCluster();
    +            testUtil = null;
    +        }
    +    }
    +
    +    public static HBaseTestingUtility getUtility(){
    +        return testUtil;
    +    }
    +
    --- End diff --
    
    I meant change the existing tests to use the GRANT/REVOKE statements (the method you added in line 258) and remove the old methods that call AccessControlClient.grant directly. 


---

[GitHub] phoenix issue #283: PHOENIX-672 Add GRANT and REVOKE commands using HBase Ac...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/phoenix/pull/283
  
    @twdsilva @aertoria @ankitsinghal @joshelser 
    Please review.
    @JamesRTaylor @apurtell FYI.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152459739
  
    --- Diff: phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java ---
    @@ -66,6 +68,46 @@ private void parseQueryThatShouldFail(String sql) throws Exception {
             }
         }
     
    +    @Test
    --- End diff --
    
    Do the permissions/user/group have to be in single quotes? 


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r153589434
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    --- End diff --
    
    This doesn't look correct, @karanmehta93. You need to use indexTableName.getPhysicalName().getBytes() here *and* then call SchemaUtil.getPhysicalTableName like this:
    
        tableName = SchemaUtil.getPhysicalTableName(indexTableName.getPhysicalName().getBytes(), indexTable.isNamespaceMapped());
    
    Otherwise, for something like an index on a view or a local index, you wouldn't get a physical table name.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152478050
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void revokePermissionsFromTable(HConnection hConnection, RevokeStatement revokeStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.revoke(hConnection, tableName, revokeStatement.getName(),
    +                null, null, Permission.Action.values());
    +    }
    --- End diff --
    
    Avoid using deprecated classes like HConnection.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152477760
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    --- End diff --
    
    I don't think it can happen. indexes should go in the same namespace in which data table is.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152644100
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java ---
    @@ -229,17 +227,12 @@ public void handleRequireAccessOnDependentTable(String request, String userName,
                         + dependentTable);
                 return;
             }
    -        if (isAutomaticGrantEnabled) {
    --- End diff --
    
    This patch removes the automatic grant option and now always keep the permissions in sync between the data table and any index tables. You can only execute a GRANT/REVOKE on the data table and not on the index table. So I don't think we need the isStrictMode option any more. Once that is removed the data table and index table permissions for all uses will always be kept in sync.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152649147
  
    --- Diff: phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java ---
    @@ -66,6 +68,46 @@ private void parseQueryThatShouldFail(String sql) throws Exception {
             }
         }
     
    +    @Test
    +    public void testParseGrantQuery() throws Exception {
    +
    +        String sql0 = "GRANT 'RX' ON SYSTEM.\"SEQUENCE\" TO 'user'";
    +        parseQuery(sql0);
    +        String sql1 = "GRANT 'RWXCA' ON TABLE some_table0 TO 'user0'";
    +        parseQuery(sql1);
    +        String sql2 = "GRANT 'RWX' ON some_table1 TO 'user1'";
    +        parseQuery(sql2);
    +        String sql3 = "GRANT 'CA' ON SCHEMA some_schema2 TO 'user2'";
    +        parseQuery(sql3);
    +        String sql4 = "GRANT 'RXW' ON some_table3 TO GROUP 'group3'";
    +        parseQuery(sql4);
    +        String sql5 = "GRANT 'RXW' ON \"some_schema5\".\"some_table5\" TO GROUP 'group5'";
    +        parseQuery(sql5);
    +        String sql6 = "GRANT 'RWA' TO 'user6'";
    +        parseQuery(sql6);
    +        String sql7 = "GRANT 'A' TO GROUP 'group7'";
    +        parseQuery(sql7);
    --- End diff --
    
    HBase API logs a message and ignores it if any other character is found apart from 'RWXCA'. If same character is passed again, it will put up the same string in the hbase:acl table.
    
     An example of this is
    `GRANT 'RXXRRRRCAAAA' to 'user11'`
    results in
    ` hbase:acl   column=l:user11, timestamp=1511376020328, value=RXXRRRRCAAAA`                                                                                                 
    
     A simple check should be to limit the size of string to 5 characters. Does that seem fine?


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152685262
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java ---
    @@ -229,17 +227,12 @@ public void handleRequireAccessOnDependentTable(String request, String userName,
                         + dependentTable);
                 return;
             }
    -        if (isAutomaticGrantEnabled) {
    --- End diff --
    
    > Oh, I see.. here we are just talking about removing a config, not the automatic grant flow in the coprocessor. I'm fine with either having these configs with suitable default or removing the configs completely
    
    Yes the code flow will still remain and the scenario that you have explained is a valid one. I will remove both the configs from the code path everywhere. 


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152650744
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java ---
    @@ -229,17 +227,12 @@ public void handleRequireAccessOnDependentTable(String request, String userName,
                         + dependentTable);
                 return;
             }
    -        if (isAutomaticGrantEnabled) {
    --- End diff --
    
    I don't get the idea of the strict mode option. If this is on, then the grants for secondary indexes would not be done, essentially breaking things. We already have enough config options, so I'd be in favor of removing this one.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152644103
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    --- End diff --
    
    Yes, I have been working on it. Will commit it here soon. Current approach is to log an error message for all the tables whose permission assignment has failed. Does that seem fine?


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 closed the pull request at:

    https://github.com/apache/phoenix/pull/283


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152479671
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    --- End diff --
    
    you can have a common method to give you set of physical tables on which revoke and grant is required.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152644779
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    --- End diff --
    
    Yes, it tries to resolve the schema immediately and throws an exception if Schema is not found. 
    `return new SchemaResolver(connection, SchemaUtil.normalizeIdentifier(schema), updateCacheImmediately: true);`


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152477915
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    --- End diff --
    
    you could use indexTable.getPhysicalName api instead.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152480644
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/TablesNotInSyncException.java ---
    @@ -0,0 +1,16 @@
    +package org.apache.phoenix.schema;
    +
    +import org.apache.phoenix.exception.SQLExceptionCode;
    +import org.apache.phoenix.exception.SQLExceptionInfo;
    +
    +import java.sql.SQLException;
    +
    +public class TablesNotInSyncException extends SQLException {
    +    private static final long serialVersionUID = 1L;
    +    private static SQLExceptionCode code = SQLExceptionCode.TABLES_NOT_IN_SYNC;
    +
    +    public TablesNotInSyncException(String table1, String table2, String diff) {
    +        super(new SQLExceptionInfo.Builder(code).setMessage("Table: " + table1 + " and Table: " + table2 + " differ in " + diff).build().toString(), code.getSQLState(), code.getErrorCode());
    +    }
    +
    +}
    --- End diff --
    
    we can add some comments here.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152457942
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    --- End diff --
    
    Can this ever happen (base table is namespace mapped but index table is not)? 


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152445087
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java ---
    @@ -229,17 +227,12 @@ public void handleRequireAccessOnDependentTable(String request, String userName,
                         + dependentTable);
                 return;
             }
    -        if (isAutomaticGrantEnabled) {
    --- End diff --
    
    What is the isStrictMode option? 


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152646903
  
    --- Diff: phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java ---
    @@ -66,6 +68,46 @@ private void parseQueryThatShouldFail(String sql) throws Exception {
             }
         }
     
    +    @Test
    --- End diff --
    
    Yes. permissions/user/group are treated as literals and table name is treated as identifier.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152442952
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java ---
    @@ -229,17 +227,12 @@ public void handleRequireAccessOnDependentTable(String request, String userName,
                         + dependentTable);
                 return;
             }
    -        if (isAutomaticGrantEnabled) {
    --- End diff --
    
    Does it make sense to also remove the isStrictMode option? We should always keep the index and data table permissions in sync.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152459035
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    --- End diff --
    
    Not sure if this check is needed as we always create the view index table for multi-tenant tables.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152644332
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    --- End diff --
    
    Yeah that's a good idea. Let me refactor the code.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r153283989
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    --- End diff --
    
    @ankitsinghal 
    I will file a JIRA for SHOW GRANTS and work on it. As of now, the code will log an error message for all the tables whose permission assignment has failed. Does that seem a good way of exception handling?


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152483233
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java ---
    @@ -210,25 +98,20 @@ public Void run() throws Exception {
                         return null;
                     }
                 });
    -            verifyAllowed(createSchema(schemaName), regularUser);
    +            verifyAllowed(createSchema(schemaName), regularUser1);
                 // Unprivileged user cannot drop a schema
    -            verifyDenied(dropSchema(schemaName), unprivilegedUser);
    -            verifyDenied(createSchema(schemaName), unprivilegedUser);
    +            verifyDenied(dropSchema(schemaName), AccessDeniedException.class, unprivilegedUser);
    +            verifyDenied(createSchema(schemaName), AccessDeniedException.class, unprivilegedUser);
     
    -            verifyAllowed(dropSchema(schemaName), regularUser);
    +            verifyAllowed(dropSchema(schemaName), regularUser1);
             } finally {
                 revokeAll();
             }
         }
     
         @Test
    -    public void testAutomaticGrantDisabled() throws Throwable{
    -        testIndexAndView(false);
    -    }
    --- End diff --
    
    Why is this removed?


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152480973
  
    --- Diff: phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java ---
    @@ -66,6 +68,46 @@ private void parseQueryThatShouldFail(String sql) throws Exception {
             }
         }
     
    +    @Test
    +    public void testParseGrantQuery() throws Exception {
    +
    +        String sql0 = "GRANT 'RX' ON SYSTEM.\"SEQUENCE\" TO 'user'";
    +        parseQuery(sql0);
    +        String sql1 = "GRANT 'RWXCA' ON TABLE some_table0 TO 'user0'";
    +        parseQuery(sql1);
    +        String sql2 = "GRANT 'RWX' ON some_table1 TO 'user1'";
    +        parseQuery(sql2);
    +        String sql3 = "GRANT 'CA' ON SCHEMA some_schema2 TO 'user2'";
    +        parseQuery(sql3);
    +        String sql4 = "GRANT 'RXW' ON some_table3 TO GROUP 'group3'";
    +        parseQuery(sql4);
    +        String sql5 = "GRANT 'RXW' ON \"some_schema5\".\"some_table5\" TO GROUP 'group5'";
    +        parseQuery(sql5);
    +        String sql6 = "GRANT 'RWA' TO 'user6'";
    +        parseQuery(sql6);
    +        String sql7 = "GRANT 'A' TO GROUP 'group7'";
    +        parseQuery(sql7);
    --- End diff --
    
    Do we have validation on permission type like fail when permission is other than 'RWXCA' at parsing side?


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152645002
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    --- End diff --
    
    Sure. Will refactor.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152646351
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void revokePermissionsFromTable(HConnection hConnection, RevokeStatement revokeStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.revoke(hConnection, tableName, revokeStatement.getName(),
    +                null, null, Permission.Action.values());
    +    }
    --- End diff --
    
    I will look into the alternate. Will refactor.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152645916
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    --- End diff --
    
    Theoretically yes, This is more of a sanity check. I don't know if you have ever encountered cases where the index table failed to get created. Since Phoenix operations are not atomic, we can end up in a partial state. The HBase Perms API doesn't check if table exists or not, so it will put up an entry even if the table doesn't exist. Let me know your thoughts and we can decide.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r153685552
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,124 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState changePermissions(ChangePermsStatement changePermsStatement) throws SQLException {
    +
    +        logger.info(changePermsStatement.toString());
    +
    +        try(HBaseAdmin admin = connection.getQueryServices().getAdmin()) {
    +            ClusterConnection clusterConnection = (ClusterConnection) admin.getConnection();
    +
    +            if (changePermsStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!changePermsStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(changePermsStatement.getSchemaName(), connection);
    +                }
    +
    +                changePermsOnSchema(clusterConnection, changePermsStatement);
    +            } else if (changePermsStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(changePermsStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT or REVOKE permissions on INDEX TABLES or VIEWS");
    +                }
    +
    +                changePermsOnTables(clusterConnection, admin, changePermsStatement, inputTable);
    +            } else {
    +
    +                changePermsOnUser(clusterConnection, changePermsStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void changePermsOnSchema(ClusterConnection clusterConnection, ChangePermsStatement changePermsStatement) throws Throwable {
    +        if(changePermsStatement.isGrantStatement()) {
    +            AccessControlClient.grant(clusterConnection, changePermsStatement.getSchemaName(), changePermsStatement.getName(), changePermsStatement.getPermsList());
    +        } else {
    +            AccessControlClient.revoke(clusterConnection, changePermsStatement.getSchemaName(), changePermsStatement.getName(), Permission.Action.values());
    +        }
    +    }
    +
    +    private void changePermsOnTables(ClusterConnection clusterConnection, HBaseAdmin admin, ChangePermsStatement changePermsStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getPhysicalName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        changePermsOnTable(clusterConnection, changePermsStatement, tableName);
    +
    +        boolean schemaInconsistency = false;
    +        List<PTable> inconsistentTables = null;
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                schemaInconsistency = true;
    +                if(inconsistentTables == null) {
    +                    inconsistentTables = new ArrayList<>();
    +                }
    +                inconsistentTables.add(indexTable);
    +                continue;
    +            }
    +            logger.info("Updating permissions for Index Table: " +
    +                    indexTable.getName() + " Base Table: " + inputTable.getName());
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getPhysicalName().getBytes(), indexTable.isNamespaceMapped());
    +            changePermsOnTable(clusterConnection, changePermsStatement, tableName);
    +        }
    +
    +        if(schemaInconsistency) {
    +            for(PTable table : inconsistentTables) {
    +                logger.error("Fail to propagate permissions to Index Table: " + table.getName());
    +            }
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    inconsistentTables.get(0).getTableName().getString(), "Namespace properties");
    +        }
    +
    +        if(inputTable.isMultiTenant()) {
    --- End diff --
    
    non multi-tenant tables could also have views which have indexes, so you need to check if the view index table exists for them and then keep the perms in sync if it does.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152642518
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java ---
    @@ -0,0 +1,635 @@
    +/*
    + * 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 com.google.common.base.Throwables;
    +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.HBaseTestingUtility;
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.security.AccessDeniedException;
    +import org.apache.hadoop.hbase.security.User;
    +import org.apache.hadoop.hbase.security.access.AccessControlClient;
    +import org.apache.hadoop.hbase.security.access.Permission;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.query.BaseTest;
    +import org.apache.phoenix.query.QueryConstants;
    +import org.apache.phoenix.query.QueryServices;
    +import org.junit.After;
    +import org.junit.BeforeClass;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +
    +import java.io.IOException;
    +import java.lang.reflect.UndeclaredThrowableException;
    +import java.security.PrivilegedExceptionAction;
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Statement;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertNotNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.fail;
    +
    +@RunWith(Parameterized.class)
    +public class BasePermissionsIT extends BaseTest {
    +
    +    private static final Log LOG = LogFactory.getLog(BasePermissionsIT.class);
    +
    +    static String SUPERUSER;
    +
    +    static HBaseTestingUtility testUtil;
    +    static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
    +            "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION"));
    +
    +    static final Set<String> PHOENIX_SYSTEM_TABLES_IDENTIFIERS = new HashSet<>(Arrays.asList(
    +            "SYSTEM.\"CATALOG\"", "SYSTEM.\"SEQUENCE\"", "SYSTEM.\"STATS\"", "SYSTEM.\"FUNCTION\""));
    +
    +    static final String SYSTEM_SEQUENCE_IDENTIFIER =
    +            QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\"" + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE+ "\"";
    +
    +    static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
    +            "SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION"));
    +
    +    // Create Multiple users so that we can use Hadoop UGI to run tasks as various users
    +    // Permissions can be granted or revoke by superusers and admins only
    +    // DON'T USE HADOOP UserGroupInformation class to create testing users since HBase misses some of its functionality
    +    // Instead use org.apache.hadoop.hbase.security.User class for testing purposes.
    +
    +    // Super User has all the access
    +    User superUser1 = null;
    +    User superUser2 = null;
    +
    +    // Regular users are granted and revoked permissions as needed
    +    User regularUser1 = null;
    +    User regularUser2 = null;
    +    User regularUser3 = null;
    +    User regularUser4 = null;
    +
    +    // Group User is equivalent of regular user but inside a group
    +    // Permissions can be granted to group should affect this user
    +    static final String GROUP_SYSTEM_ACCESS = "group_system_access";
    +    User groupUser = null;
    +
    +    // Unpriviledged User doesn't have any access and is denied for every action
    +    User unprivilegedUser = null;
    +
    +    static final int NUM_RECORDS = 5;
    +
    +    boolean isNamespaceMapped;
    +
    +    public BasePermissionsIT(final boolean isNamespaceMapped) throws Exception {
    +        this.isNamespaceMapped = isNamespaceMapped;
    +    }
    +
    +    @BeforeClass
    +    public static void doSetup() throws Exception {
    +        SUPERUSER = System.getProperty("user.name");
    +    }
    +
    +    void startNewMiniCluster() throws Exception {
    +        startNewMiniCluster(new Configuration());
    +    }
    +    
    +    void startNewMiniCluster(Configuration overrideConf) throws Exception{
    +        if (null != testUtil) {
    +            testUtil.shutdownMiniCluster();
    +            testUtil = null;
    +        }
    +
    +        testUtil = new HBaseTestingUtility();
    +
    +        Configuration config = testUtil.getConfiguration();
    +        enablePhoenixHBaseAuthorization(config);
    +        configureNamespacesOnServer(config);
    +        configureRandomHMasterPort(config);
    +        if (overrideConf != null) {
    +            config.addResource(overrideConf);
    +        }
    +
    +        testUtil.startMiniCluster(1);
    +        initializeUsers(testUtil.getConfiguration());
    +    }
    +
    +    private void initializeUsers(Configuration configuration) {
    +
    +        superUser1 = User.createUserForTesting(configuration, SUPERUSER, new String[0]);
    +        superUser2 = User.createUserForTesting(configuration, "superUser2", new String[0]);
    +
    +        regularUser1 = User.createUserForTesting(configuration, "regularUser1", new String[0]);
    +        regularUser2 = User.createUserForTesting(configuration, "regularUser2", new String[0]);
    +        regularUser3 = User.createUserForTesting(configuration, "regularUser3", new String[0]);
    +        regularUser4 = User.createUserForTesting(configuration, "regularUser4", new String[0]);
    +
    +        groupUser = User.createUserForTesting(testUtil.getConfiguration(), "groupUser", new String[] {GROUP_SYSTEM_ACCESS});
    +
    +        unprivilegedUser = User.createUserForTesting(configuration, "unprivilegedUser", new String[0]);
    +    }
    +
    +    private void configureRandomHMasterPort(Configuration config) {
    +        // Avoid multiple clusters trying to bind the master's info port (16010)
    +        config.setInt(HConstants.MASTER_INFO_PORT, -1);
    +    }
    +
    +    void enablePhoenixHBaseAuthorization(Configuration config) {
    +        config.set("hbase.superuser", SUPERUSER + "," + "superUser2");
    +        config.set("hbase.security.authorization", Boolean.TRUE.toString());
    +        config.set("hbase.security.exec.permission.checks", Boolean.TRUE.toString());
    +        config.set("hbase.coprocessor.master.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +        config.set("hbase.coprocessor.region.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +        config.set("hbase.coprocessor.regionserver.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +
    +        config.set(QueryServices.PHOENIX_ACLS_ENABLED,"true");
    +
    +        config.set("hbase.regionserver.wal.codec", "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec");
    +    }
    +
    +    void configureNamespacesOnServer(Configuration conf) {
    +        conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
    +    }
    +
    +    @Parameterized.Parameters(name = "isNamespaceMapped={0}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean> data() {
    +        return Arrays.asList(false, true);
    +    }
    +
    +    @After
    +    public void cleanup() throws Exception {
    +        if (testUtil != null) {
    +            testUtil.shutdownMiniCluster();
    +            testUtil = null;
    +        }
    +    }
    +
    +    public static HBaseTestingUtility getUtility(){
    +        return testUtil;
    +    }
    +
    --- End diff --
    
    This is the base class for all Permission Tests. Checkout the method on Line 258. The original tests have been just refactored; GrantRevokePermissionsIT contains the required tests for this JIRA.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152460399
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java ---
    @@ -0,0 +1,635 @@
    +/*
    + * 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 com.google.common.base.Throwables;
    +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.HBaseTestingUtility;
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.hadoop.hbase.security.AccessDeniedException;
    +import org.apache.hadoop.hbase.security.User;
    +import org.apache.hadoop.hbase.security.access.AccessControlClient;
    +import org.apache.hadoop.hbase.security.access.Permission;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.query.BaseTest;
    +import org.apache.phoenix.query.QueryConstants;
    +import org.apache.phoenix.query.QueryServices;
    +import org.junit.After;
    +import org.junit.BeforeClass;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +
    +import java.io.IOException;
    +import java.lang.reflect.UndeclaredThrowableException;
    +import java.security.PrivilegedExceptionAction;
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Statement;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertNotNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.fail;
    +
    +@RunWith(Parameterized.class)
    +public class BasePermissionsIT extends BaseTest {
    +
    +    private static final Log LOG = LogFactory.getLog(BasePermissionsIT.class);
    +
    +    static String SUPERUSER;
    +
    +    static HBaseTestingUtility testUtil;
    +    static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
    +            "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION"));
    +
    +    static final Set<String> PHOENIX_SYSTEM_TABLES_IDENTIFIERS = new HashSet<>(Arrays.asList(
    +            "SYSTEM.\"CATALOG\"", "SYSTEM.\"SEQUENCE\"", "SYSTEM.\"STATS\"", "SYSTEM.\"FUNCTION\""));
    +
    +    static final String SYSTEM_SEQUENCE_IDENTIFIER =
    +            QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\"" + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE+ "\"";
    +
    +    static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
    +            "SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION"));
    +
    +    // Create Multiple users so that we can use Hadoop UGI to run tasks as various users
    +    // Permissions can be granted or revoke by superusers and admins only
    +    // DON'T USE HADOOP UserGroupInformation class to create testing users since HBase misses some of its functionality
    +    // Instead use org.apache.hadoop.hbase.security.User class for testing purposes.
    +
    +    // Super User has all the access
    +    User superUser1 = null;
    +    User superUser2 = null;
    +
    +    // Regular users are granted and revoked permissions as needed
    +    User regularUser1 = null;
    +    User regularUser2 = null;
    +    User regularUser3 = null;
    +    User regularUser4 = null;
    +
    +    // Group User is equivalent of regular user but inside a group
    +    // Permissions can be granted to group should affect this user
    +    static final String GROUP_SYSTEM_ACCESS = "group_system_access";
    +    User groupUser = null;
    +
    +    // Unpriviledged User doesn't have any access and is denied for every action
    +    User unprivilegedUser = null;
    +
    +    static final int NUM_RECORDS = 5;
    +
    +    boolean isNamespaceMapped;
    +
    +    public BasePermissionsIT(final boolean isNamespaceMapped) throws Exception {
    +        this.isNamespaceMapped = isNamespaceMapped;
    +    }
    +
    +    @BeforeClass
    +    public static void doSetup() throws Exception {
    +        SUPERUSER = System.getProperty("user.name");
    +    }
    +
    +    void startNewMiniCluster() throws Exception {
    +        startNewMiniCluster(new Configuration());
    +    }
    +    
    +    void startNewMiniCluster(Configuration overrideConf) throws Exception{
    +        if (null != testUtil) {
    +            testUtil.shutdownMiniCluster();
    +            testUtil = null;
    +        }
    +
    +        testUtil = new HBaseTestingUtility();
    +
    +        Configuration config = testUtil.getConfiguration();
    +        enablePhoenixHBaseAuthorization(config);
    +        configureNamespacesOnServer(config);
    +        configureRandomHMasterPort(config);
    +        if (overrideConf != null) {
    +            config.addResource(overrideConf);
    +        }
    +
    +        testUtil.startMiniCluster(1);
    +        initializeUsers(testUtil.getConfiguration());
    +    }
    +
    +    private void initializeUsers(Configuration configuration) {
    +
    +        superUser1 = User.createUserForTesting(configuration, SUPERUSER, new String[0]);
    +        superUser2 = User.createUserForTesting(configuration, "superUser2", new String[0]);
    +
    +        regularUser1 = User.createUserForTesting(configuration, "regularUser1", new String[0]);
    +        regularUser2 = User.createUserForTesting(configuration, "regularUser2", new String[0]);
    +        regularUser3 = User.createUserForTesting(configuration, "regularUser3", new String[0]);
    +        regularUser4 = User.createUserForTesting(configuration, "regularUser4", new String[0]);
    +
    +        groupUser = User.createUserForTesting(testUtil.getConfiguration(), "groupUser", new String[] {GROUP_SYSTEM_ACCESS});
    +
    +        unprivilegedUser = User.createUserForTesting(configuration, "unprivilegedUser", new String[0]);
    +    }
    +
    +    private void configureRandomHMasterPort(Configuration config) {
    +        // Avoid multiple clusters trying to bind the master's info port (16010)
    +        config.setInt(HConstants.MASTER_INFO_PORT, -1);
    +    }
    +
    +    void enablePhoenixHBaseAuthorization(Configuration config) {
    +        config.set("hbase.superuser", SUPERUSER + "," + "superUser2");
    +        config.set("hbase.security.authorization", Boolean.TRUE.toString());
    +        config.set("hbase.security.exec.permission.checks", Boolean.TRUE.toString());
    +        config.set("hbase.coprocessor.master.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +        config.set("hbase.coprocessor.region.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +        config.set("hbase.coprocessor.regionserver.classes",
    +                "org.apache.hadoop.hbase.security.access.AccessController");
    +
    +        config.set(QueryServices.PHOENIX_ACLS_ENABLED,"true");
    +
    +        config.set("hbase.regionserver.wal.codec", "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec");
    +    }
    +
    +    void configureNamespacesOnServer(Configuration conf) {
    +        conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
    +    }
    +
    +    @Parameterized.Parameters(name = "isNamespaceMapped={0}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean> data() {
    +        return Arrays.asList(false, true);
    +    }
    +
    +    @After
    +    public void cleanup() throws Exception {
    +        if (testUtil != null) {
    +            testUtil.shutdownMiniCluster();
    +            testUtil = null;
    +        }
    +    }
    +
    +    public static HBaseTestingUtility getUtility(){
    +        return testUtil;
    +    }
    +
    --- End diff --
    
    Instead of directly calling the AccessControlClient can the tests just use the GRANT or REVOKE statements?


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r153605197
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    --- End diff --
    
    The code that I wrote also works because it takes in the original name and uses its namespace mapped property to determine the Physical table name. 
    If you see the code for `getPhysicalName()` in `PTableImpl`, it is the `SchemaUtil.getPhysicalHBaseTableName(schemaName, tableName, isNamespaceMapped)`
    The simplest and cleaner option in this case is to use `TableName.valueOf(indexTableName.getPhysicalName().getBytes())`. 
    I was trying to avoid using the HBase API's directly and use the SchemaUtil methods. What do you suggest?
    @JamesRTaylor 


---

[GitHub] phoenix issue #283: PHOENIX-672 Add GRANT and REVOKE commands using HBase Ac...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on the issue:

    https://github.com/apache/phoenix/pull/283
  
    ping @ankitsinghal 
    
    Karan's patch removes the automatic grant option and always keeps the index and data tables in sync. Can you please take a look as well.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152642780
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java ---
    @@ -210,25 +98,20 @@ public Void run() throws Exception {
                         return null;
                     }
                 });
    -            verifyAllowed(createSchema(schemaName), regularUser);
    +            verifyAllowed(createSchema(schemaName), regularUser1);
                 // Unprivileged user cannot drop a schema
    -            verifyDenied(dropSchema(schemaName), unprivilegedUser);
    -            verifyDenied(createSchema(schemaName), unprivilegedUser);
    +            verifyDenied(dropSchema(schemaName), AccessDeniedException.class, unprivilegedUser);
    +            verifyDenied(createSchema(schemaName), AccessDeniedException.class, unprivilegedUser);
     
    -            verifyAllowed(dropSchema(schemaName), regularUser);
    +            verifyAllowed(dropSchema(schemaName), regularUser1);
             } finally {
                 revokeAll();
             }
         }
     
         @Test
    -    public void testAutomaticGrantDisabled() throws Throwable{
    -        testIndexAndView(false);
    -    }
    --- End diff --
    
    I removed the option of AUTOMATIC_GRANT and set it defaulted to true.


---

[GitHub] phoenix issue #283: PHOENIX-672 Add GRANT and REVOKE commands using HBase Ac...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/phoenix/pull/283
  
    @twdsilva @ankitsinghal @JamesRTaylor 
    
    Updates:
    Added ChangePermsStatement that combines the Grant and Revoke Statement
    Added test for Multitenant tables with view indexes
    Updated grammar to reject permission strings with length more than 5
    General code refactoring based on comments and added new comments


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152663200
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    --- End diff --
    
    This is more like sanity check again.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r153582948
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    --- End diff --
    
    We need HBase TableName object, whereas getPhysicalName api returns PName, so it wont help us here.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152656117
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java ---
    @@ -229,17 +227,12 @@ public void handleRequireAccessOnDependentTable(String request, String userName,
                         + dependentTable);
                 return;
             }
    -        if (isAutomaticGrantEnabled) {
    --- End diff --
    
    Oh, I see.. here we are just talking about removing a config, not the automatic grant flow in the coprocessor. I'm fine with either way, having these configs with suitable default or removing the configs completely.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152457161
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java ---
    @@ -229,17 +227,12 @@ public void handleRequireAccessOnDependentTable(String request, String userName,
                         + dependentTable);
                 return;
             }
    -        if (isAutomaticGrantEnabled) {
    --- End diff --
    
    In handleRequireAccessOnDependentTable if the isStrictMode boolean is set to false then the automatic grant code path is skipped.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152648189
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java ---
    @@ -229,17 +227,12 @@ public void handleRequireAccessOnDependentTable(String request, String userName,
                         + dependentTable);
                 return;
             }
    -        if (isAutomaticGrantEnabled) {
    --- End diff --
    
    @karanmehta93 
    Strict mode:- It will check permissions for dependent tables as well. For eg, If a user who has all access on data table is creating an index, then we need to ensure that all others users of data table can also access a new index table.
    AutomaticGrant:- It will automatically grant required permissions to dependent table users.
    
    @twdsilva , what about the case when a new index is been created?
    Purpose of the automatic grant:- let's say there are three users A and B have READ permission on the data-table and user C has RWC permission on data-table. so if user B creates an index, then we need to ensure that user A and C should also be able to read the index and C should be able to write to this Index and can drop the index also. so we will give only the required permission to the users of data-table on the index table. So, Access should propagate like this. 
    
    user | access data table | access on index table | with Automatic grant(access on index table will change like this) | comments
    -- | -- | -- | -- | --
    A | RAX | no access | RX | RX will be given on index table
    B | RX | RWXC | RWXC | no grant will happen
    C | RWXAC | no access | RWCX | read ,write and create will be given so that it can read/write to index table and drop as well.
    
    
    



---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152481907
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java ---
    @@ -229,17 +227,12 @@ public void handleRequireAccessOnDependentTable(String request, String userName,
                         + dependentTable);
                 return;
             }
    -        if (isAutomaticGrantEnabled) {
    --- End diff --
    
    I think , we can keep this option. as it will help admin if he doesn't want to see the permissions problems on the dependent table and wants to manage the permission by himself once the table is created(with downtime) and don't want to rely on the automatic grant.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152478348
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    +                }
    +                revokePermissionsFromSchema(hConnection, revokeStatement);
    +
    +            } else if (revokeStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(revokeStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot REVOKE permissions from INDEX TABLES or VIEWS");
    +                }
    +                revokePermissionsFromTables(hConnection, revokeStatement, inputTable);
    +
    +            } else {
    +                revokePermissionsFromUser(hConnection, revokeStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +
    +    private void revokePermissionsFromTables(HConnection hConnection, RevokeStatement revokeStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Revoking perms from IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            revokePermissionsFromTable(hConnection, revokeStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    --- End diff --
    
    Admin is not closed.


---

[GitHub] phoenix issue #283: PHOENIX-672 Add GRANT and REVOKE commands using HBase Ac...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/phoenix/pull/283
  
    @samarthjain Please have a look.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152646448
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/TablesNotInSyncException.java ---
    @@ -0,0 +1,16 @@
    +package org.apache.phoenix.schema;
    +
    +import org.apache.phoenix.exception.SQLExceptionCode;
    +import org.apache.phoenix.exception.SQLExceptionInfo;
    +
    +import java.sql.SQLException;
    +
    +public class TablesNotInSyncException extends SQLException {
    +    private static final long serialVersionUID = 1L;
    +    private static SQLExceptionCode code = SQLExceptionCode.TABLES_NOT_IN_SYNC;
    +
    +    public TablesNotInSyncException(String table1, String table2, String diff) {
    +        super(new SQLExceptionInfo.Builder(code).setMessage("Table: " + table1 + " and Table: " + table2 + " differ in " + diff).build().toString(), code.getSQLState(), code.getErrorCode());
    +    }
    +
    +}
    --- End diff --
    
    Sure. Will refactor.


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152457669
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    +            }
    +
    +        } catch (SQLException e) {
    +            // Bubble up the SQL Exception
    +            throw e;
    +        } catch (Throwable throwable) {
    +            // Wrap around other exceptions to PhoenixIOException (Ex: org.apache.hadoop.hbase.security.AccessDeniedException)
    +            throw ServerUtil.parseServerException(throwable);
    +        }
    +
    +        return new MutationState(0, 0, connection);
    +    }
    +
    +    private void grantPermissionsToTables(HConnection hConnection, GrantStatement grantStatement, PTable inputTable) throws Throwable {
    +
    +        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
    +                (inputTable.getName().getBytes(), inputTable.isNamespaceMapped());
    +
    +        grantPermissionsToTable(hConnection, grantStatement, tableName);
    +
    +        for(PTable indexTable : inputTable.getIndexes()) {
    +            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
    +            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
    +                continue;
    +            }
    +            logger.info("Granting " + Arrays.toString(grantStatement.getPermsList()) +
    +                    " perms to IndexTable: " + indexTable.getName() + " BaseTable: " + inputTable.getName());
    +            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
    +                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                        indexTable.getTableName().getString(), "Namespace properties");
    +            }
    +            tableName = SchemaUtil.getPhysicalTableName(indexTable.getName().getBytes(), indexTable.isNamespaceMapped());
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
    +        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
    +        boolean viewIndexTableExists = connection.getQueryServices().getAdmin().tableExists(tableName);
    +        if(!viewIndexTableExists && inputTable.isMultiTenant()) {
    +            logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
    +            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
    +                    Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
    +        }
    +        if(viewIndexTableExists) {
    +            grantPermissionsToTable(hConnection, grantStatement, tableName);
    +        }
    +
    +    }
    +
    +    private void grantPermissionsToTable(HConnection hConnection, GrantStatement grantStatement, org.apache.hadoop.hbase.TableName tableName)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, tableName, grantStatement.getName(),
    +                null, null, grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToSchema(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getSchemaName(), grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    private void grantPermissionsToUser(HConnection hConnection, GrantStatement grantStatement)
    +            throws Throwable {
    +        AccessControlClient.grant(hConnection, grantStatement.getName(), grantStatement.getPermsList());
    +    }
    +
    +    public MutationState revokePermission(RevokeStatement revokeStatement) throws SQLException {
    +
    +        StringBuffer revokePermLog = new StringBuffer();
    +        revokePermLog.append("Revoke Permissions requested for user/group: " + revokeStatement.getName());
    +        if (revokeStatement.getSchemaName() != null) {
    +            revokePermLog.append(" for Schema: " + revokeStatement.getSchemaName());
    +        } else if (revokeStatement.getTableName() != null) {
    +            revokePermLog.append(" for Table: " + revokeStatement.getTableName());
    +        }
    +        logger.info(revokePermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (revokeStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!revokeStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(revokeStatement.getSchemaName(), connection);
    --- End diff --
    
    Does FromCompiler.getResolverForSchema actually check to see if the schema exists?


---

[GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H...

Posted by ankitsinghal <gi...@git.apache.org>.
Github user ankitsinghal commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/283#discussion_r152483142
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
    @@ -4168,4 +4176,197 @@ public MutationState useSchema(UseSchemaStatement useSchemaStatement) throws SQL
             }
             return new MutationState(0, 0, connection);
         }
    +
    +    public MutationState grantPermission(GrantStatement grantStatement) throws SQLException {
    +
    +        StringBuffer grantPermLog = new StringBuffer();
    +        grantPermLog.append("Grant Permissions requested for user/group: " + grantStatement.getName());
    +        if (grantStatement.getSchemaName() != null) {
    +            grantPermLog.append(" for Schema: " + grantStatement.getSchemaName());
    +        } else if (grantStatement.getTableName() != null) {
    +            grantPermLog.append(" for Table: " + grantStatement.getTableName());
    +        }
    +        grantPermLog.append(" Permissions: " + Arrays.toString(grantStatement.getPermsList()));
    +        logger.info(grantPermLog.toString());
    +
    +        HConnection hConnection = connection.getQueryServices().getAdmin().getConnection();
    +
    +        try {
    +            if (grantStatement.getSchemaName() != null) {
    +                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
    +                if(!grantStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
    +                    FromCompiler.getResolverForSchema(grantStatement.getSchemaName(), connection);
    +                }
    +                grantPermissionsToSchema(hConnection, grantStatement);
    +
    +            } else if (grantStatement.getTableName() != null) {
    +                PTable inputTable = PhoenixRuntime.getTable(connection,
    +                        SchemaUtil.normalizeFullTableName(grantStatement.getTableName().toString()));
    +                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
    +                    throw new AccessDeniedException("Cannot GRANT permissions on INDEX TABLES or VIEWS");
    +                }
    +                grantPermissionsToTables(hConnection, grantStatement, inputTable);
    +
    +            } else {
    +                grantPermissionsToUser(hConnection, grantStatement);
    --- End diff --
    
    If for some reason grant doesn't succeed for all the tables. so do we have plan to give construct like "SHOW GRANTS" or something to the user to know what all grants are still there for the user or on the table.


---