You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by pr...@apache.org on 2014/05/17 02:07:37 UTC

git commit: SENTRY-150: Support various SHOW commands (Sravya Tirukkovalur via Prasad Mujumdar)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master b1d4dde5e -> b3ee46494


SENTRY-150: Support various SHOW commands (Sravya Tirukkovalur via Prasad Mujumdar)


Project: http://git-wip-us.apache.org/repos/asf/incubator-sentry/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-sentry/commit/b3ee4649
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/b3ee4649
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/b3ee4649

Branch: refs/heads/master
Commit: b3ee46494335fd4a8ab0c444ec1508b1b5962593
Parents: b1d4dde
Author: Prasad Mujumdar <pr...@cloudera.com>
Authored: Fri May 16 17:07:07 2014 -0700
Committer: Prasad Mujumdar <pr...@cloudera.com>
Committed: Fri May 16 17:07:07 2014 -0700

----------------------------------------------------------------------
 .../hive/ql/exec/SentryGrantRevokeTask.java     | 253 +++++++++----------
 .../SentryHiveAuthorizationTaskFactoryImpl.java |   6 +-
 sentry-tests/sentry-tests-hive/pom.xml          |   6 +
 .../tests/e2e/hive/TestDatabaseProvider.java    | 173 ++++++++++++-
 .../e2e/hive/hiveserver/AbstractHiveServer.java |  16 +-
 5 files changed, 306 insertions(+), 148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b3ee4649/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
index 2776eae..b6f39ad 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
@@ -17,14 +17,10 @@ package org.apache.hadoop.hive.ql.exec;
  * limitations under the License.
  */
 
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.SentryHiveConstants;
@@ -49,22 +45,21 @@ import org.apache.sentry.SentryUserException;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
-import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.core.common.Subject;
-import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
+import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
+import org.apache.sentry.provider.db.service.thrift.TSentryRole;
 import org.apache.sentry.service.thrift.SentryServiceClientFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Splitter;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Set;
 
-// TODO remove this suppress
-@SuppressWarnings("unused")
 public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable {
   private static final Logger LOG = LoggerFactory
       .getLogger(SentryGrantRevokeTask.class);
@@ -129,7 +124,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
             server, work.getRevokeDesc());
       }
       if (work.getShowGrantDesc() != null) {
-        return processShowGrantDDL(conf, console, subject.getName(), subjectGroups,
+        return processShowGrantDDL(conf, console, sentryClient, subject.getName(), subjectGroups,
             work.getShowGrantDesc());
       }
       if (work.getGrantRevokeRoleDDL() != null) {
@@ -171,8 +166,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
     this.subjectGroups = subjectGroups;
   }
 
-  @VisibleForTesting
-  static int processRoleDDL(HiveConf conf, LogHelper console,
+  private int processRoleDDL(HiveConf conf, LogHelper console,
       SentryPolicyServiceClient sentryClient, String subject,
       Set<String> subjectGroups, HiveAuthzBinding hiveAuthzBinding, RoleDDLDesc desc)
           throws SentryUserException {
@@ -190,27 +184,21 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
         sentryClient.dropRole(subject, subjectGroups, name);
         return RETURN_CODE_SUCCESS;
       } else if (operation.equals(RoleDDLDesc.RoleOperation.SHOW_ROLE_GRANT)) {
+        Set<TSentryRole> roles;
         PrincipalType principalType = desc.getPrincipalType();
         if (principalType != PrincipalType.GROUP) {
           String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + principalType;
           throw new HiveException(msg);
         }
-        throw new AssertionError("TODO");
-        // TODO once retrieval API is implemented this can be implemented
-//        List<String> roles = sentryClient.getRoles(name);
-//        if (!roles.isEmpty()) {
-//          Path resFile = new Path(desc.getResFile());
-//          FileSystem fs = resFile.getFileSystem(conf);
-//          outStream = fs.create(resFile);
-//          for (String role : roles) {
-//            outStream.writeBytes("role name:" + role);
-//            outStream.write(terminator);
-//          }
-//          outStream = close(outStream);
-//        }
-//        return RETURN_CODE_SUCCESS;
-      } else {
-        throw new HiveException("Unkown role operation "
+        roles = sentryClient.listRolesByGroupName(subject, subjectGroups, desc.getName() );
+        writeToFile(writeRoleGrantsInfo(roles), desc.getResFile());
+        return RETURN_CODE_SUCCESS;
+      } else if(operation.equals(RoleDDLDesc.RoleOperation.SHOW_ROLES)) {
+      Set<TSentryRole> roles = sentryClient.listRoles(subject, subjectGroups);
+      writeToFile(writeRolesInfo(roles), desc.getResFile());
+      return RETURN_CODE_SUCCESS;
+    } else {
+        throw new HiveException("Unknown role operation "
             + operation.getOperationName());
       }
     } catch (HiveException e) {
@@ -220,26 +208,25 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
       LOG.warn(msg, e);
       console.printError(msg);
       return RETURN_CODE_FAILURE;
-//    } catch (IOException e) {
-//      String msg = "IO Error in role operation " + e.getMessage();
-//      LOG.info(msg, e);
-//      console.printError(msg);
-//      return RETURN_CODE_FAILURE;
+    } catch (IOException e) {
+      String msg = "IO Error in role operation " + e.getMessage();
+      LOG.info(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
     } finally {
       closeQuiet(outStream);
     }
   }
 
-  @VisibleForTesting
-  static int processGrantDDL(HiveConf conf, LogHelper console,
+  private int processGrantDDL(HiveConf conf, LogHelper console,
       SentryPolicyServiceClient sentryClient, String subject,
       Set<String> subjectGroups, String server, GrantDesc desc) throws SentryUserException {
     return processGrantRevokeDDL(console, sentryClient, subject, subjectGroups,
         server, true, desc.getPrincipals(), desc.getPrivileges(), desc.getPrivilegeSubjectDesc());
   }
 
-  @VisibleForTesting
-  static int processRevokeDDL(HiveConf conf, LogHelper console,
+
+  private int processRevokeDDL(HiveConf conf, LogHelper console,
       SentryPolicyServiceClient sentryClient, String subject,
       Set<String> subjectGroups, String server, RevokeDesc desc) throws SentryUserException {
     return processGrantRevokeDDL(console, sentryClient, subject, subjectGroups,
@@ -247,103 +234,47 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
         desc.getPrivilegeSubjectDesc());
   }
 
-  @VisibleForTesting
-  static int processShowGrantDDL(HiveConf conf, LogHelper console, String subject,
-      Set<String> subjectGroups, ShowGrantDesc desc) {
-    DataOutputStream outStream = null;
+  private int processShowGrantDDL(HiveConf conf, LogHelper console, SentryPolicyServiceClient sentryClient,
+      String subject, Set<String> subjectGroups, ShowGrantDesc desc) throws SentryUserException{
+    PrincipalDesc principalDesc = desc.getPrincipalDesc();
+    PrivilegeObjectDesc hiveObjectDesc = desc.getHiveObj();
+    String principalName = principalDesc.getName();
+
+    Set<TSentryPrivilege> privileges;
+
     try {
-      Path resFile = new Path(desc.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-      PrincipalDesc principalDesc = desc.getPrincipalDesc();
-      PrivilegeObjectDesc hiveObjectDesc = desc.getHiveObj();
-      String principalName = principalDesc.getName();
-      List<String> columns = desc.getColumns();
-      if (columns != null && !columns.isEmpty()) {
-        throw new HiveException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
-      }
       if (hiveObjectDesc == null) {
-        // TDOD get users from somewhere?
-        List<String> users = Collections.emptyList();
-        if (users != null && users.size() > 0) {
-          boolean first = true;
-          Collections.sort(users);
-          for (String usr : users) {
-            if (!first) {
-              outStream.write(terminator);
-            } else {
-              first = false;
-            }
-            // TODO write grant info
-          }
-        }
+        privileges = sentryClient.listPrivilegesByRoleName(subject, subjectGroups, principalName);
+        writeToFile(writeGrantInfo(privileges, principalName), desc.getResFile());
+        return RETURN_CODE_SUCCESS;
       } else {
-        if (hiveObjectDesc.getPartSpec() != null) {
-          throw new HiveException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
-        }
-        String obj = hiveObjectDesc.getObject();
-        String dbName = null;
-        String tableName = null;
-        if (hiveObjectDesc.getTable()) {
-          DatabaseTable dbTable = parseDBTable(obj);
-          dbName = dbTable.getDatabase();
-          tableName = dbTable.getTable();
-        } else {
-          dbName = hiveObjectDesc.getObject();
-        }
-        if (hiveObjectDesc.getTable()) {
-          // show table level privileges
-          // TODO
-          List<String> tbls = Collections.emptyList();
-          if (tbls != null && tbls.size() > 0) {
-            boolean first = true;
-            Collections.sort(tbls);
-            for (String tbl : tbls) {
-              if (!first) {
-                outStream.write(terminator);
-              } else {
-                first = false;
-              }
-              // TODO write grant info
-            }
-          }
-        } else {
-          // show database level privileges
-          // TODO
-          List<String> dbs = Collections.emptyList();
-          if (dbs != null && dbs.size() > 0) {
-            boolean first = true;
-            Collections.sort(dbs);
-            for (String db : dbs) {
-              if (!first) {
-                outStream.write(terminator);
-              } else {
-                first = false;
-              }
-              // TODO write grant info
-            }
-          }
-        }
+        throw new AssertionError("TODO: SHOW GRANT role <roleName> on <objectType> <privilegeLevel>");
       }
-      outStream = close(outStream);
-    } catch (HiveException e) {
-      String msg = "Error in show grant operation " + e.getMessage();
-      LOG.warn(msg, e);
-      console.printError(msg);
-      return RETURN_CODE_FAILURE;
     } catch (IOException e) {
       String msg = "IO Error in show grant " + e.getMessage();
       LOG.info(msg, e);
       console.printError(msg);
       return RETURN_CODE_FAILURE;
+    }
+
+  }
+  private void writeToFile(String data, String file) throws IOException {
+    Path resFile = new Path(file);
+    FileSystem fs = resFile.getFileSystem(conf);
+    FSDataOutputStream out = fs.create(resFile);
+    try {
+      if (data != null && !data.isEmpty()) {
+        OutputStreamWriter writer = new OutputStreamWriter(out, "UTF-8");
+        writer.write(data);
+        writer.write((char) terminator);
+        writer.flush();
+      }
     } finally {
-      closeQuiet(outStream);
+      closeQuiet(out);
     }
-    return RETURN_CODE_SUCCESS;
   }
 
-  @VisibleForTesting
-  static int processGrantRevokeRoleDDL(HiveConf conf, LogHelper console,
+  private int processGrantRevokeRoleDDL(HiveConf conf, LogHelper console,
       SentryPolicyServiceClient sentryClient, String subject, Set<String> subjectGroups,
       GrantRevokeRoleDDL desc) throws SentryUserException {
     try {
@@ -374,6 +305,70 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
     return RETURN_CODE_SUCCESS;
   }
 
+  static String writeGrantInfo(Set<TSentryPrivilege> privileges, String roleName) {
+    if (privileges == null || privileges.isEmpty()) {
+      return "";
+    }
+    StringBuilder builder = new StringBuilder();
+
+    for (TSentryPrivilege privilege : privileges) {
+
+      appendNonNull(builder, privilege.getDbName(), true);
+      appendNonNull(builder, privilege.getTableName());
+      appendNonNull(builder, null);//getPartValues()
+      appendNonNull(builder, null);//getColumnName()
+      appendNonNull(builder, roleName);//getPrincipalName()
+      appendNonNull(builder, "ROLE");//getPrincipalType()
+      appendNonNull(builder, privilege.getAction());
+      appendNonNull(builder, false);//isGrantOption()
+      appendNonNull(builder, privilege.getCreateTime() * 1000L);
+      appendNonNull(builder, privilege.getGrantorPrincipal());
+    }
+    LOG.info("builder.toString(): " + builder.toString());
+    return builder.toString();
+  }
+
+  static String writeRoleGrantsInfo(Set<TSentryRole> roleGrants) {
+    if (roleGrants == null || roleGrants.isEmpty()) {
+      return "";
+    }
+    StringBuilder builder = new StringBuilder();
+    for (TSentryRole roleGrant : roleGrants) {
+      appendNonNull(builder, roleGrant.getRoleName(), true);
+      appendNonNull(builder, false);//isGrantOption()
+      appendNonNull(builder, null);//roleGrant.getGrantTime() * 1000L
+      appendNonNull(builder, roleGrant.getGrantorPrincipal());
+    }
+    return builder.toString();
+  }
+
+  static String writeRolesInfo(Set<TSentryRole> roles) {
+    if (roles == null || roles.isEmpty()) {
+      return "";
+    }
+    StringBuilder builder = new StringBuilder();
+    for (TSentryRole roleGrant : roles) {
+      appendNonNull(builder, roleGrant.getRoleName(), true);
+    }
+    return builder.toString();
+  }
+
+  static StringBuilder appendNonNull(StringBuilder builder, Object value) {
+    return appendNonNull(builder, value, false);
+  }
+
+  static StringBuilder appendNonNull(StringBuilder builder, Object value, boolean firstColumn) {
+    if (!firstColumn) {
+      builder.append((char)separator);
+    } else if (builder.length() > 0) {
+      builder.append((char)terminator);
+    }
+    if (value != null) {
+      builder.append(value);
+    }
+    return builder;
+  }
+
   private static int processGrantRevokeDDL(LogHelper console,
       SentryPolicyServiceClient sentryClient, String subject,
       Set<String> subjectGroups, String server,

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b3ee4649/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
index 2e7a9d9..9ca11ae 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
@@ -210,6 +210,8 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
     }
     String principalName = BaseSemanticAnalyzer.unescapeIdentifier(principal.getChild(0).getText());
     PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
+
+    //Column privileges and Partition privileges are not supported by Sentry
     if (ast.getChildCount() > 1) {
       ASTNode child = (ASTNode) ast.getChild(1);
       if (child.getToken().getType() == HiveParser.TOK_PRIV_OBJECT_COL) {
@@ -230,10 +232,6 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
       }
     }
 
-    if (privHiveObj == null) {
-      throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
-    }
-
     ShowGrantDesc showGrant = new ShowGrantDesc(resultFile.toString(),
         principalDesc, privHiveObj, null);
     return createTask(new DDLWork(inputs, outputs, showGrant));

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b3ee4649/sentry-tests/sentry-tests-hive/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/pom.xml b/sentry-tests/sentry-tests-hive/pom.xml
index 0005a53..1d0e169 100644
--- a/sentry-tests/sentry-tests-hive/pom.xml
+++ b/sentry-tests/sentry-tests-hive/pom.xml
@@ -219,6 +219,12 @@ limitations under the License.
           </exclusion>
       </exclusions>
     </dependency>
+      <dependency>
+          <groupId>org.hamcrest</groupId>
+          <artifactId>hamcrest-all</artifactId>
+          <scope>test</scope>
+          <version>1.3</version>
+      </dependency>
   </dependencies>
   <build>
     <plugins>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b3ee4649/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
index 51bb623..176acee 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
@@ -17,12 +17,8 @@
 
 package org.apache.sentry.tests.e2e.hive;
 
-import java.io.File;
-import java.sql.Connection;
-import java.sql.Statement;
-import java.util.Map;
-import java.util.concurrent.TimeoutException;
-
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -35,11 +31,24 @@ import org.apache.sentry.service.thrift.ServiceConstants.ClientConfig;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory;
 import org.junit.After;
+import static org.junit.Assert.assertThat;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-import com.google.common.io.Files;
+import static org.hamcrest.Matchers.*;
+
+import java.io.File;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertTrue;
 
 public class TestDatabaseProvider extends AbstractTestWithHiveServer {
   protected static final String SERVER_HOST = "localhost";
@@ -58,7 +67,7 @@ public class TestDatabaseProvider extends AbstractTestWithHiveServer {
     policyFile = new PolicyFile();
     properties.put(HiveServerFactory.AUTHZ_PROVIDER_BACKEND, SimpleDBProviderBackend.class.getName());
     properties.put(ConfVars.HIVE_AUTHORIZATION_TASK_FACTORY.varname,
-        SentryHiveAuthorizationTaskFactoryImpl.class.getName());
+      SentryHiveAuthorizationTaskFactoryImpl.class.getName());
     properties.put(ServerConfig.SECURITY_MODE, ServerConfig.SECURITY_MODE_NONE);
     properties.put(ServerConfig.ADMIN_GROUPS, ADMINGROUP);
     properties.put(ServerConfig.RPC_ADDRESS, SERVER_HOST);
@@ -98,6 +107,19 @@ public class TestDatabaseProvider extends AbstractTestWithHiveServer {
     }
   }
 
+  /**
+   * This test is only used for manual testing of beeline with Sentry Service
+   * @throws Exception
+   */
+  @Ignore
+  @Test
+  public void beelineTest() throws Exception{
+    policyFile
+        .setUserGroupMapping(StaticUserGroup.getStaticMapping())
+        .write(context.getPolicyFile());
+    while(true) {}
+  }
+
   @Test
   public void testBasic() throws Exception {
     policyFile
@@ -131,4 +153,137 @@ public class TestDatabaseProvider extends AbstractTestWithHiveServer {
     statement.close();
     connection.close();
   }
+
+  /**
+   * SHOW ROLES
+   * @throws Exception
+   */
+  @Test
+  public void testShowRoles() throws Exception {
+    policyFile
+        .setUserGroupMapping(StaticUserGroup.getStaticMapping())
+        .write(context.getPolicyFile());
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE role1");
+    statement.execute("CREATE ROLE role2");
+    ResultSet resultSet = statement.executeQuery("SHOW ROLES");
+    Set<String> roles = new HashSet<String>();
+    while ( resultSet.next()) {
+      roles.add(resultSet.getString(1));
+    }
+    assertThat(roles.size(), is(new Integer(2)));
+    assertTrue(roles.contains("role1"));
+    assertTrue(roles.contains("role2"));
+    statement.close();
+    connection.close();
+  }
+
+  /**
+   * SHOW ROLE GRANT GROUP groupName
+   * @throws Exception
+   */
+  @Test
+  public void testShowRolesByGroup() throws Exception {
+    policyFile
+        .setUserGroupMapping(StaticUserGroup.getStaticMapping())
+        .write(context.getPolicyFile());
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE role1");
+    statement.execute("CREATE ROLE role2");
+    statement.execute("CREATE ROLE role3");
+    statement.execute("GRANT ROLE role1 to GROUP " + ADMINGROUP);
+
+    ResultSet resultSet = statement.executeQuery("SHOW ROLE GRANT GROUP " + ADMINGROUP);
+    Set<String> roles = new HashSet<String>();
+    while ( resultSet.next()) {
+      assertThat(resultSet.getString(1), equalToIgnoringCase("role1"));
+      assertThat(resultSet.getBoolean(2), is(new Boolean("False")));
+      //Create time is not tested
+      //assertThat(resultSet.getLong(3), is(new Long(0)));
+      assertThat(resultSet.getString(4), equalToIgnoringCase(ADMIN1));
+    }
+    statement.close();
+    connection.close();
+  }
+
+  /**
+   * SHOW GRANT ROLE roleName
+   * @throws Exception
+   */
+  @Test
+  public void testShowPrivilegesByRole() throws Exception {
+    policyFile
+        .setUserGroupMapping(StaticUserGroup.getStaticMapping())
+        .write(context.getPolicyFile());
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE role1");
+    statement.execute("CREATE ROLE role2");
+    statement.execute("GRANT SELECT ON TABLE t1 TO ROLE role1");
+
+    ResultSet resultSet = statement.executeQuery("SHOW GRANT ROLE role1");
+    while ( resultSet.next()) {
+      assertThat(resultSet.getString(1), equalToIgnoringCase("default"));
+      assertThat(resultSet.getString(2), equalToIgnoringCase("t1"));
+      assertThat(resultSet.getString(3), equalToIgnoringCase(""));//partition
+      assertThat(resultSet.getString(4), equalToIgnoringCase(""));//column
+      assertThat(resultSet.getString(5), equalToIgnoringCase("role1"));//principalName
+      assertThat(resultSet.getString(6), equalToIgnoringCase("role"));//principalType
+      assertThat(resultSet.getString(7), equalToIgnoringCase("select"));
+      assertThat(resultSet.getBoolean(8), is(new Boolean("False")));//grantOption
+      //Create time is not tested
+      //assertThat(resultSet.getLong(9), is(new Long(0)));
+      assertThat(resultSet.getString(10), equalToIgnoringCase(ADMIN1));//grantor
+    }
+    statement.close();
+    connection.close();
+  }
+
+  /**
+   * SHOW GRANT ROLE roleName ON OBJECT PRIVILEGE not supported yet
+   * @throws Exception
+   */
+  @Test
+  public void testShowPrivilegesByRoleAndObject() throws Exception {
+    policyFile
+        .setUserGroupMapping(StaticUserGroup.getStaticMapping())
+        .write(context.getPolicyFile());
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE role1");
+    statement.execute("GRANT SELECT ON TABLE t1 TO ROLE role1");
+
+    try {
+      ResultSet resultSet = statement.executeQuery("SHOW GRANT ROLE role1 ON TABLE tab1");
+      assertTrue("Expected an exception", false);
+    } catch(SQLException e) {
+      statement.close();
+      connection.close();
+    }
+  }
+  /**
+   * SHOW CURRENT ROLE not supported yet
+   * @throws Exception
+   */
+  @Ignore
+  @Test
+  public void testShowCurrentRole() throws Exception {
+    policyFile
+        .setUserGroupMapping(StaticUserGroup.getStaticMapping())
+        .write(context.getPolicyFile());
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE role1");
+    statement.execute("SET ROLE role1");
+
+    try {
+      ResultSet resultSet = statement.executeQuery("SHOW CURRENT ROLE");
+      assertTrue("Expected an exception", false);
+    } catch(SQLException e) {
+      statement.close();
+      connection.close();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b3ee4649/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/AbstractHiveServer.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/AbstractHiveServer.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/AbstractHiveServer.java
index be6eafc..ed95d48 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/AbstractHiveServer.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/AbstractHiveServer.java
@@ -16,18 +16,21 @@
  */
 package org.apache.sentry.tests.e2e.hive.hiveserver;
 
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.util.concurrent.TimeoutException;
-
+import com.google.common.base.Strings;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Strings;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.concurrent.TimeoutException;
 
 public abstract class AbstractHiveServer implements HiveServer {
+  private static final Logger LOGGER = LoggerFactory.getLogger(InternalHiveServer.class);
+
 
   private static final String LINK_FAILURE_SQL_STATE = "08S01";
 
@@ -40,6 +43,7 @@ public abstract class AbstractHiveServer implements HiveServer {
     this.configuration = configuration;
     this.hostname = hostname;
     this.port = port;
+    LOGGER.info("Hive Server2 configured on host: " + hostname + " port:" + port);
   }
 
   @Override