Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id A9D66200D41 for ; Wed, 22 Nov 2017 07:28:33 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id A834C160BFD; Wed, 22 Nov 2017 06:28:33 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id C898D160BDA for ; Wed, 22 Nov 2017 07:28:32 +0100 (CET) Received: (qmail 57265 invoked by uid 500); 22 Nov 2017 06:28:31 -0000 Mailing-List: contact dev-help@phoenix.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@phoenix.apache.org Delivered-To: mailing list dev@phoenix.apache.org Received: (qmail 57254 invoked by uid 99); 22 Nov 2017 06:28:31 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 22 Nov 2017 06:28:31 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 1F4C4F5C3D; Wed, 22 Nov 2017 06:28:31 +0000 (UTC) From: ankitsinghal To: dev@phoenix.apache.org Reply-To: dev@phoenix.apache.org References: In-Reply-To: Subject: [GitHub] phoenix pull request #283: PHOENIX-672 Add GRANT and REVOKE commands using H... Content-Type: text/plain Message-Id: <20171122062831.1F4C4F5C3D@git1-us-west.apache.org> Date: Wed, 22 Nov 2017 06:28:31 +0000 (UTC) archived-at: Wed, 22 Nov 2017 06:28:33 -0000 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. ---