You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by co...@apache.org on 2017/10/27 09:22:22 UTC

[1/6] sentry git commit: SENTRY-2014: incorrect handling of HDFS paths with multiple forward slashes (Vadim Spector, reviewed by Sergio Pena and Arjun Mishra)

Repository: sentry
Updated Branches:
  refs/heads/akolb-cli 99f03c35d -> 8be62797b


SENTRY-2014: incorrect handling of HDFS paths with multiple forward slashes (Vadim Spector, reviewed by Sergio Pena and Arjun Mishra)


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

Branch: refs/heads/akolb-cli
Commit: 7dbadfe859ae7caf06fcd3aca6e81d20636515d8
Parents: 1f77657
Author: Vadim Spector <vs...@cloudera.com>
Authored: Tue Oct 24 09:57:13 2017 -0700
Committer: Vadim Spector <vs...@cloudera.com>
Committed: Tue Oct 24 09:57:13 2017 -0700

----------------------------------------------------------------------
 .../sentry/core/common/utils/PathUtils.java     |  9 ++++++
 .../org/apache/sentry/hdfs/PathsUpdate.java     |  9 ++++--
 .../org/apache/sentry/hdfs/TestPathsUpdate.java | 32 +++++++++++++++-----
 .../db/service/persistent/SentryStore.java      |  3 +-
 .../service/thrift/NotificationProcessor.java   |  3 +-
 5 files changed, 44 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/7dbadfe8/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
index 40c9595..cef8bd7 100644
--- a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
+++ b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
@@ -211,4 +211,13 @@ public class PathUtils {
     return uriPath.toUri().toString();
   }
 
+  /**
+   * Split path into elements.
+   * May evolve to do something smarter, e.g. path canonicalization,
+   * but for now simple split on "/" is sufficient.
+   */
+  public static String[] splitPath(String path) {
+    return (path != null) ? path.split("/+") : null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/7dbadfe8/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PathsUpdate.java
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PathsUpdate.java b/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PathsUpdate.java
index 5ff2dc2..c9ecc40 100644
--- a/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PathsUpdate.java
+++ b/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PathsUpdate.java
@@ -166,8 +166,13 @@ public class PathsUpdate implements Updateable.Update {
       throw new SentryMalformedPathException("Path part of uri does not seem right, was expecting a non empty path" +
               ": path = " + uriPath + ", uri=" + uri);
     }
-    // Remove leading slash
-    return uriPath.substring(1);
+    // Reduce multiple consecutive forward slashes to one.
+    // It's probably a rare case, so use indexOf() before expensive regex.
+    if (uriPath.indexOf("//") >= 0) {
+      uriPath = uriPath.replaceAll("//*", "/");
+    }
+    // Remove leading and trailing slashes
+    return StringUtils.strip(uriPath, "/");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/sentry/blob/7dbadfe8/sentry-hdfs/sentry-hdfs-common/src/test/java/org/apache/sentry/hdfs/TestPathsUpdate.java
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-common/src/test/java/org/apache/sentry/hdfs/TestPathsUpdate.java b/sentry-hdfs/sentry-hdfs-common/src/test/java/org/apache/sentry/hdfs/TestPathsUpdate.java
index c1a8a74..6c8ed2b 100644
--- a/sentry-hdfs/sentry-hdfs-common/src/test/java/org/apache/sentry/hdfs/TestPathsUpdate.java
+++ b/sentry-hdfs/sentry-hdfs-common/src/test/java/org/apache/sentry/hdfs/TestPathsUpdate.java
@@ -42,14 +42,30 @@ public class TestPathsUpdate {
 
   @Test
   public void testPositiveParsePath() throws SentryMalformedPathException {
-    String result = PathsUpdate.parsePath("hdfs://hostname.test.com:8020/path");
-    Assert.assertTrue("Parsed path is unexpected", result.equals("path"));
-
-    result = PathsUpdate.parsePath("hdfs://hostname.test.com/path");
-    Assert.assertTrue("Parsed path is unexpected", result.equals("path"));
-
-    result = PathsUpdate.parsePath("hdfs:///path");
-    Assert.assertTrue("Parsed path is unexpected", result.equals("path"));
+    String urls[] = {
+      "hdfs://hostname.test.com:8020/path1/path2/path3",
+      // double slashes
+      "hdfs://hostname.test.com:8020//path1/path2/path3",
+      "hdfs://hostname.test.com:8020/path1//path2/path3",
+      "hdfs://hostname.test.com:8020/path1/path2//path3",
+      "hdfs://hostname.test.com:8020/path1/path2//path3",
+      "hdfs://hostname.test.com:8020/path1/path2//path3//",
+      // triple slashes
+      "hdfs://hostname.test.com:8020///path1/path2/path3",
+      "hdfs://hostname.test.com:8020/path1///path2/path3",
+      "hdfs://hostname.test.com:8020/path1/path2///path3",
+      "hdfs://hostname.test.com:8020/path1/path2/path3///",
+      // no port
+      "hdfs://hostname.test.com/path1/path2/path3",
+      // no host
+      "hdfs:///path1/path2/path3"
+    };
+    String path = "path1/path2/path3";
+      
+    for (String url : urls) {
+      String result = PathsUpdate.parsePath(url);
+      Assert.assertEquals("Unexpected path in " + url, path, result);
+    }
   }
 
   @Test(expected = SentryMalformedPathException.class)

http://git-wip-us.apache.org/repos/asf/sentry/blob/7dbadfe8/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
index f4d84d2..0cd6e48 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
@@ -49,6 +49,7 @@ import org.apache.sentry.core.common.exception.SentryInvalidInputException;
 import org.apache.sentry.core.common.exception.SentryNoSuchObjectException;
 import org.apache.sentry.core.common.exception.SentrySiteConfigurationException;
 import org.apache.sentry.core.common.exception.SentryUserException;
+import org.apache.sentry.core.common.utils.PathUtils;
 import org.apache.sentry.core.common.utils.SentryConstants;
 import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
@@ -2777,7 +2778,7 @@ public class SentryStore {
       String  objName = authzToPaths.getAuthzObjName();
       // Convert path strings to list of components
       for (String path: authzToPaths.getPathStrings()) {
-        String[] pathComponents = path.split("/");
+        String[] pathComponents = PathUtils.splitPath(path);
         List<String> paths = new ArrayList<>(pathComponents.length);
         Collections.addAll(paths, pathComponents);
         pathUpdate.applyAddChanges(objName, Collections.singletonList(paths));

http://git-wip-us.apache.org/repos/asf/sentry/blob/7dbadfe8/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
index 07a7db4..d92f23e 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
@@ -38,6 +38,7 @@ import org.apache.sentry.binding.metastore.messaging.json.SentryJSONMessageDeser
 import org.apache.sentry.core.common.exception.SentryInvalidHMSEventException;
 import org.apache.sentry.core.common.exception.SentryInvalidInputException;
 import org.apache.sentry.core.common.exception.SentryNoSuchObjectException;
+import org.apache.sentry.core.common.utils.PathUtils;
 import org.apache.sentry.hdfs.PathsUpdate;
 import org.apache.sentry.hdfs.PermissionsUpdate;
 import org.apache.sentry.hdfs.SentryMalformedPathException;
@@ -111,7 +112,7 @@ final class NotificationProcessor {
    * @return list of components, e.g. [foo, bar]
    */
   private static List<String> splitPath(String path) {
-    return (Lists.newArrayList(path.split("/")));
+    return Lists.newArrayList(PathUtils.splitPath(path));
   }
 
   /**


[3/6] sentry git commit: SENTRY-2013 - Align the SentryGenericServiceClient and SentryPolicyServiceClient a bit more closely - Signed off by Kalyan.

Posted by co...@apache.org.
SENTRY-2013 - Align the SentryGenericServiceClient and SentryPolicyServiceClient a bit more closely
 - Signed off by Kalyan.


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

Branch: refs/heads/akolb-cli
Commit: cfa8abb07f460a9021b70694c1d63a0892db01d9
Parents: 6d44cfd
Author: Colm O hEigeartaigh <co...@apache.org>
Authored: Wed Oct 25 12:11:13 2017 +0100
Committer: Colm O hEigeartaigh <co...@apache.org>
Committed: Wed Oct 25 12:11:13 2017 +0100

----------------------------------------------------------------------
 .../DefaultSentryAccessController.java          |  2 +-
 .../hive/ql/exec/SentryGrantRevokeTask.java     |  2 +-
 .../authz/DefaultSentryAccessController.java    |  2 +-
 .../sentry/kafka/binding/KafkaAuthBinding.java  |  6 +--
 .../sentry/sqoop/binding/SqoopAuthBinding.java  |  8 ++--
 .../provider/db/generic/UpdatableCache.java     |  2 +-
 .../thrift/SentryGenericServiceClient.java      | 14 +++---
 .../SentryGenericServiceClientDefaultImpl.java  | 12 ++---
 .../db/generic/tools/SentryConfigToolSolr.java  |  2 +-
 .../tools/command/GenericShellCommand.java      |  6 +--
 .../thrift/SentryPolicyServiceClient.java       |  2 +-
 .../SentryPolicyServiceClientDefaultImpl.java   |  2 +-
 .../db/tools/command/hive/HiveShellCommand.java |  2 +-
 .../TestAuditLogForSentryGenericService.java    |  8 ++--
 .../TestSentryGenericServiceIntegration.java    | 48 ++++++++++----------
 .../generic/tools/TestSentryConfigToolSolr.java |  4 +-
 .../db/generic/tools/TestSentryShellKafka.java  |  2 +-
 .../db/generic/tools/TestSentryShellSolr.java   |  2 +-
 .../db/generic/tools/TestSentryShellSqoop.java  |  2 +-
 .../thrift/TestSentryPolicyServiceClient.java   |  4 +-
 .../thrift/TestSentryServiceClientPool.java     |  6 +--
 .../thrift/TestSentryServiceFailureCase.java    |  2 +-
 .../thrift/TestSentryServiceIntegration.java    |  8 ++--
 .../TestSentryServiceWithInvalidMsgSize.java    | 10 ++--
 .../provider/db/tools/TestSentryShellHive.java  |  2 +-
 .../thrift/SentryServiceIntegrationBase.java    |  2 +-
 .../e2e/dbprovider/TestConcurrentClients.java   |  2 +-
 .../metastore/SentryPolicyProviderForDb.java    |  2 +-
 .../e2e/dbprovider/TestConcurrentClients.java   |  2 +-
 .../metastore/SentryPolicyProviderForDb.java    |  2 +-
 .../e2e/kafka/AbstractKafkaSentryTestBase.java  |  2 +-
 .../sentry/tests/e2e/kafka/TestAuthorize.java   |  2 +-
 .../AbstractSolrSentryTestWithDbProvider.java   |  4 +-
 .../e2e/sqoop/AbstractSqoopSentryTestBase.java  |  2 +-
 34 files changed, 90 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryAccessController.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryAccessController.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryAccessController.java
index 456c8d6..fed483f 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryAccessController.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryAccessController.java
@@ -156,7 +156,7 @@ public class DefaultSentryAccessController extends SentryHiveAccessController {
     List<String> roles = new ArrayList<String>();
     try {
       sentryClient = getSentryClient();
-      roles = convert2RoleList(sentryClient.listRoles(authenticator.getUserName()));
+      roles = convert2RoleList(sentryClient.listAllRoles(authenticator.getUserName()));
     } catch (SentryAccessDeniedException e) {
       HiveOperation hiveOp = HiveOperation.SHOW_ROLES;
       executeOnFailureHooks(hiveOp, e);

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/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 96e57f1..61052cf 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
@@ -242,7 +242,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
         writeToFile(writeRoleGrantsInfo(roles), desc.getResFile());
         return RETURN_CODE_SUCCESS;
       } else if(operation.equals(RoleDDLDesc.RoleOperation.SHOW_ROLES)) {
-        Set<TSentryRole> roles = sentryClient.listRoles(subject);
+        Set<TSentryRole> roles = sentryClient.listAllRoles(subject);
         writeToFile(writeRolesInfo(roles), desc.getResFile());
         return RETURN_CODE_SUCCESS;
       } else if(operation.equals(RoleDDLDesc.RoleOperation.SHOW_CURRENT_ROLE)) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryAccessController.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryAccessController.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryAccessController.java
index 0bfa52b..0f93b42 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryAccessController.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryAccessController.java
@@ -153,7 +153,7 @@ public class DefaultSentryAccessController extends SentryHiveAccessController {
     List<String> roles = new ArrayList<String>();
     try {
       sentryClient = getSentryClient();
-      roles = convert2RoleList(sentryClient.listRoles(authenticator.getUserName()));
+      roles = convert2RoleList(sentryClient.listAllRoles(authenticator.getUserName()));
     } catch (SentryAccessDeniedException e) {
       HiveOperation hiveOp = HiveOperation.SHOW_ROLES;
       executeOnFailureHooks(hiveOp, e);

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-binding/sentry-binding-kafka/src/main/java/org/apache/sentry/kafka/binding/KafkaAuthBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-kafka/src/main/java/org/apache/sentry/kafka/binding/KafkaAuthBinding.java b/sentry-binding/sentry-binding-kafka/src/main/java/org/apache/sentry/kafka/binding/KafkaAuthBinding.java
index 660e66f..908c80f 100644
--- a/sentry-binding/sentry-binding-kafka/src/main/java/org/apache/sentry/kafka/binding/KafkaAuthBinding.java
+++ b/sentry-binding/sentry-binding-kafka/src/main/java/org/apache/sentry/kafka/binding/KafkaAuthBinding.java
@@ -291,7 +291,7 @@ public class KafkaAuthBinding {
     execute(new Command<Void>() {
       @Override
       public Void run(SentryGenericServiceClient client) throws Exception {
-        client.addRoleToGroups(
+        client.grantRoleToGroups(
             requestorName, role, COMPONENT_NAME, groups);
         return null;
       }
@@ -432,7 +432,7 @@ public class KafkaAuthBinding {
       @Override
       public Void run(SentryGenericServiceClient client) throws Exception {
         for (String role : roles) {
-          tSentryPrivileges.addAll(client.listPrivilegesByRoleName(
+          tSentryPrivileges.addAll(client.listAllPrivilegesByRoleName(
                 requestorName, role, COMPONENT_NAME, instanceName));
         }
         return null;
@@ -501,7 +501,7 @@ public class KafkaAuthBinding {
       @Override
       public Void run(SentryGenericServiceClient client) throws Exception {
         for (String role : roles) {
-          final Set<TSentryPrivilege> rolePrivileges = client.listPrivilegesByRoleName(
+          final Set<TSentryPrivilege> rolePrivileges = client.listAllPrivilegesByRoleName(
               requestorName, role, COMPONENT_NAME, instanceName);
           final scala.collection.immutable.Set<TSentryPrivilege> rolePrivilegesScala =
               scala.collection.JavaConverters.asScalaSetConverter(rolePrivileges).asScala().toSet();

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopAuthBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopAuthBinding.java b/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopAuthBinding.java
index 79ec477..b7cbd32 100644
--- a/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopAuthBinding.java
+++ b/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopAuthBinding.java
@@ -241,9 +241,9 @@ public class SqoopAuthBinding {
       public Set<TSentryPrivilege> run(SentryGenericServiceClient client)
           throws Exception {
         if (resource == null) {
-          return client.listPrivilegesByRoleName(subject.getName(), role, COMPONENT_TYPE, sqoopServer.getName());
+          return client.listAllPrivilegesByRoleName(subject.getName(), role, COMPONENT_TYPE, sqoopServer.getName());
         } else if (resource.getType().equalsIgnoreCase(MResource.TYPE.SERVER.name())) {
-          return client.listPrivilegesByRoleName(subject.getName(), role, COMPONENT_TYPE, resource.getName());
+          return client.listAllPrivilegesByRoleName(subject.getName(), role, COMPONENT_TYPE, resource.getName());
         } else {
           return client.listPrivilegesByRoleName(subject.getName(), role, COMPONENT_TYPE, sqoopServer.getName(), toAuthorizable(resource));
         }
@@ -281,7 +281,7 @@ public class SqoopAuthBinding {
     execute(new Command<Void>() {
       @Override
       public Void run(SentryGenericServiceClient client) throws Exception {
-        client.addRoleToGroups(subject.getName(), role.getName(), COMPONENT_TYPE, Sets.newHashSet(group));
+        client.grantRoleToGroups(subject.getName(), role.getName(), COMPONENT_TYPE, Sets.newHashSet(group));
         return null;
       }
     });
@@ -291,7 +291,7 @@ public class SqoopAuthBinding {
     execute(new Command<Void>() {
       @Override
       public Void run(SentryGenericServiceClient client) throws Exception {
-        client.deleteRoleToGroups(subject.getName(), role.getName(), COMPONENT_TYPE, Sets.newHashSet(group));
+        client.revokeRoleFromGroups(subject.getName(), role.getName(), COMPONENT_TYPE, Sets.newHashSet(group));
         return null;
       }
     });

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/UpdatableCache.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/UpdatableCache.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/UpdatableCache.java
index d20710f..edf0934 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/UpdatableCache.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/UpdatableCache.java
@@ -106,7 +106,7 @@ public final class UpdatableCache implements TableCache, AutoCloseable {
       for (TSentryRole tSentryRole : tSentryRoles) {
         final String roleName = tSentryRole.getRoleName();
         final Set<TSentryPrivilege> tSentryPrivileges =
-                client.listPrivilegesByRoleName(requestor, roleName, componentType, serviceName);
+                client.listAllPrivilegesByRoleName(requestor, roleName, componentType, serviceName);
         for (String group : tSentryRole.getGroups()) {
           Set<String> currentPrivileges = tempCache.get(group, roleName);
           if (currentPrivileges == null) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SentryGenericServiceClient.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SentryGenericServiceClient.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SentryGenericServiceClient.java
index 246d0b4..dd49952 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SentryGenericServiceClient.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SentryGenericServiceClient.java
@@ -54,25 +54,25 @@ public interface SentryGenericServiceClient extends AutoCloseable {
       String component) throws SentryUserException;
 
   /**
-   * add a sentry role to groups.
+   * Grant a sentry role to groups.
    * @param requestorUserName: user on whose behalf the request is issued
    * @param roleName: Name of the role
    * @param component: The request is issued to which component
    * @param groups: The name of groups
    * @throws SentryUserException
    */
-  void addRoleToGroups(String requestorUserName, String roleName,
+  void grantRoleToGroups(String requestorUserName, String roleName,
       String component, Set<String> groups) throws SentryUserException;
 
   /**
-   * delete a sentry role from groups.
+   * revoke a sentry role from groups.
    * @param requestorUserName: user on whose behalf the request is issued
    * @param roleName: Name of the role
    * @param component: The request is issued to which component
    * @param groups: The name of groups
    * @throws SentryUserException
    */
-  void deleteRoleToGroups(String requestorUserName, String roleName,
+  void revokeRoleFromGroups(String requestorUserName, String roleName,
       String component, Set<String> groups) throws SentryUserException;
 
   /**
@@ -104,7 +104,7 @@ public interface SentryGenericServiceClient extends AutoCloseable {
    * @param privilege
    * @throws SentryUserException
    */
-  void dropPrivilege(String requestorUserName,String component,
+  void dropPrivilege(String requestorUserName, String component,
       TSentryPrivilege privilege) throws SentryUserException;
 
   /**
@@ -155,7 +155,7 @@ public interface SentryGenericServiceClient extends AutoCloseable {
       String serviceName, List<? extends Authorizable> authorizables)
       throws SentryUserException;
 
-  Set<TSentryPrivilege> listPrivilegesByRoleName(
+  Set<TSentryPrivilege> listAllPrivilegesByRoleName(
       String requestorUserName, String roleName, String component,
       String serviceName) throws SentryUserException;
 
@@ -188,7 +188,7 @@ public interface SentryGenericServiceClient extends AutoCloseable {
    * @returns The mapping of authorize objects and TSentryPrivilegeMap(<role, set<privileges>).
    * @throws SentryUserException
    */
-  Map<String, TSentryPrivilegeMap> listPrivilegsbyAuthorizable(String component,
+  Map<String, TSentryPrivilegeMap> listPrivilegesbyAuthorizable(String component,
       String serviceName, String requestorUserName, Set<String> authorizablesSet,
       Set<String> groups, ActiveRoleSet roleSet) throws SentryUserException;
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SentryGenericServiceClientDefaultImpl.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SentryGenericServiceClientDefaultImpl.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SentryGenericServiceClientDefaultImpl.java
index 6301a6b..4cd8fd6 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SentryGenericServiceClientDefaultImpl.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SentryGenericServiceClientDefaultImpl.java
@@ -182,7 +182,7 @@ public class SentryGenericServiceClientDefaultImpl
   }
 
   /**
-   * add a sentry role to groups.
+   * Grant a sentry role to groups.
    *
    * @param requestorUserName: user on whose behalf the request is issued
    * @param roleName:          Name of the role
@@ -191,7 +191,7 @@ public class SentryGenericServiceClientDefaultImpl
    * @throws SentryUserException
    */
   @Override
-  public void addRoleToGroups(String requestorUserName, String roleName,
+  public void grantRoleToGroups(String requestorUserName, String roleName,
                               String component, Set<String> groups) throws SentryUserException {
     TAlterSentryRoleAddGroupsRequest request = new TAlterSentryRoleAddGroupsRequest();
     request.setProtocol_version(sentry_common_serviceConstants.TSENTRY_SERVICE_V2);
@@ -209,7 +209,7 @@ public class SentryGenericServiceClientDefaultImpl
   }
 
   /**
-   * delete a sentry role from groups.
+   * revoke a sentry role from groups.
    *
    * @param requestorUserName: user on whose behalf the request is issued
    * @param roleName:          Name of the role
@@ -218,7 +218,7 @@ public class SentryGenericServiceClientDefaultImpl
    * @throws SentryUserException
    */
   @Override
-  public void deleteRoleToGroups(String requestorUserName, String roleName,
+  public void revokeRoleFromGroups(String requestorUserName, String roleName,
                                  String component, Set<String> groups) throws SentryUserException {
     TAlterSentryRoleDeleteGroupsRequest request = new TAlterSentryRoleDeleteGroupsRequest();
     request.setProtocol_version(sentry_common_serviceConstants.TSENTRY_SERVICE_V2);
@@ -441,7 +441,7 @@ public class SentryGenericServiceClientDefaultImpl
   }
 
   @Override
-  public Set<TSentryPrivilege> listPrivilegesByRoleName(
+  public Set<TSentryPrivilege> listAllPrivilegesByRoleName(
     String requestorUserName, String roleName, String component,
     String serviceName) throws SentryUserException {
     return listPrivilegesByRoleName(requestorUserName, roleName, component, serviceName, null);
@@ -505,7 +505,7 @@ public class SentryGenericServiceClientDefaultImpl
    * @returns The mapping of authorize objects and TSentryPrivilegeMap(<role, set<privileges>).
    */
   @Override
-  public Map<String, TSentryPrivilegeMap> listPrivilegsbyAuthorizable(String component,
+  public Map<String, TSentryPrivilegeMap> listPrivilegesbyAuthorizable(String component,
                                                                       String serviceName, String requestorUserName, Set<String> authorizablesSet,
                                                                       Set<String> groups, ActiveRoleSet roleSet) throws SentryUserException {
 

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryConfigToolSolr.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryConfigToolSolr.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryConfigToolSolr.java
index b2664de..77d3919 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryConfigToolSolr.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryConfigToolSolr.java
@@ -124,7 +124,7 @@ public class SentryConfigToolSolr extends SentryConfigToolCommon {
         }
         LOGGER.info(dryRunMessage(importPolicy) + "Adding role: " + roleName.toLowerCase(Locale.US) + " to group: " + groupName);
         if (importPolicy) {
-          client.addRoleToGroups(requestorName, roleName, component, Sets.newHashSet(groupName));
+          client.grantRoleToGroups(requestorName, roleName, component, Sets.newHashSet(groupName));
         }
 
         for (String permission : privileges) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GenericShellCommand.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GenericShellCommand.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GenericShellCommand.java
index 5a3baad..11615ff 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GenericShellCommand.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GenericShellCommand.java
@@ -64,7 +64,7 @@ public class GenericShellCommand implements ShellCommand {
 
   public void grantRoleToGroups(String requestorName, String roleName, String groups) throws SentryUserException {
     Set<String> groupSet = Sets.newHashSet(groups.split(SentryShellCommon.GROUP_SPLIT_CHAR));
-    client.addRoleToGroups(requestorName, roleName, component, groupSet);
+    client.grantRoleToGroups(requestorName, roleName, component, groupSet);
   }
 
   public void revokePrivilegeFromRole(String requestorName, String roleName, String privilege) throws SentryUserException {
@@ -74,7 +74,7 @@ public class GenericShellCommand implements ShellCommand {
 
   public void revokeRoleFromGroups(String requestorName, String roleName, String groups) throws SentryUserException {
     Set<String> groupSet = Sets.newHashSet(groups.split(SentryShellCommon.GROUP_SPLIT_CHAR));
-    client.deleteRoleToGroups(requestorName, roleName, component, groupSet);
+    client.revokeRoleFromGroups(requestorName, roleName, component, groupSet);
   }
 
   public List<String> listRoles(String requestorName, String roleName, String group) throws SentryUserException {
@@ -97,7 +97,7 @@ public class GenericShellCommand implements ShellCommand {
 
   public List<String> listPrivileges(String requestorName, String roleName) throws SentryUserException {
     Set<TSentryPrivilege> privileges = client
-        .listPrivilegesByRoleName(requestorName, roleName, component, serviceName);
+        .listAllPrivilegesByRoleName(requestorName, roleName, component, serviceName);
 
     List<String> result = new ArrayList<>();
     if (privileges != null) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
index fb8036f..61833fc 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
@@ -56,7 +56,7 @@ public interface SentryPolicyServiceClient extends AutoCloseable {
   Set<TSentryPrivilege> listPrivilegesByRoleName(String requestorUserName, String roleName,
       List<? extends Authorizable> authorizable) throws SentryUserException;
 
-  Set<TSentryRole> listRoles(String requestorUserName) throws SentryUserException;
+  Set<TSentryRole> listAllRoles(String requestorUserName) throws SentryUserException;
 
   Set<TSentryRole> listUserRoles(String requestorUserName) throws SentryUserException;
 

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClientDefaultImpl.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClientDefaultImpl.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClientDefaultImpl.java
index b5b8f82..7ada138 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClientDefaultImpl.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClientDefaultImpl.java
@@ -238,7 +238,7 @@ public class SentryPolicyServiceClientDefaultImpl implements SentryPolicyService
   }
 
   @Override
-  public Set<TSentryRole> listRoles(String requestorUserName)
+  public Set<TSentryRole> listAllRoles(String requestorUserName)
     throws SentryUserException {
     return listRolesByGroupName(requestorUserName, null);
   }

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/HiveShellCommand.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/HiveShellCommand.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/HiveShellCommand.java
index 8451d8b..1e0692b 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/HiveShellCommand.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/HiveShellCommand.java
@@ -76,7 +76,7 @@ public class HiveShellCommand implements ShellCommand {
   public List<String> listRoles(String requestorName, String roleName, String group) throws SentryUserException {
     Set<TSentryRole> roles;
     if (StringUtils.isEmpty(group)) {
-      roles = client.listRoles(requestorName);
+      roles = client.listAllRoles(requestorName);
     } else {
       roles = client.listRolesByGroupName(requestorName, group);
     }

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/thrift/TestAuditLogForSentryGenericService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/thrift/TestAuditLogForSentryGenericService.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/thrift/TestAuditLogForSentryGenericService.java
index 6c7d22d..fbf8af3 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/thrift/TestAuditLogForSentryGenericService.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/thrift/TestAuditLogForSentryGenericService.java
@@ -139,7 +139,7 @@ public class TestAuditLogForSentryGenericService extends SentryServiceIntegratio
         }
 
         // test the audit log for add role to group, success
-        client.addRoleToGroups(requestorUserName, roleName, COMPONENT,
+        client.grantRoleToGroups(requestorUserName, roleName, COMPONENT,
             Sets.newHashSet(testGroupName));
         fieldValueMap.clear();
         fieldValueMap.put(Constants.LOG_FIELD_OPERATION, Constants.OPERATION_ADD_ROLE);
@@ -152,7 +152,7 @@ public class TestAuditLogForSentryGenericService extends SentryServiceIntegratio
 
         // test the audit log for add role to group, failed
         try {
-          client.addRoleToGroups(requestorUserName, "invalidRole", COMPONENT,
+          client.grantRoleToGroups(requestorUserName, "invalidRole", COMPONENT,
               Sets.newHashSet(testGroupName));
           fail("Exception should have been thrown");
         } catch (Exception e) {
@@ -228,7 +228,7 @@ public class TestAuditLogForSentryGenericService extends SentryServiceIntegratio
         }
 
         // test the audit log for delete role from group, success
-        client.deleteRoleToGroups(requestorUserName, roleName, COMPONENT,
+        client.revokeRoleFromGroups(requestorUserName, roleName, COMPONENT,
             Sets.newHashSet(testGroupName));
         fieldValueMap.clear();
         fieldValueMap.put(Constants.LOG_FIELD_OPERATION, Constants.OPERATION_DELETE_ROLE);
@@ -240,7 +240,7 @@ public class TestAuditLogForSentryGenericService extends SentryServiceIntegratio
         assertAuditLog(fieldValueMap);
         // test the audit log for delete role from group, failed
         try {
-          client.deleteRoleToGroups(requestorUserName, "invalidRole", COMPONENT,
+          client.revokeRoleFromGroups(requestorUserName, "invalidRole", COMPONENT,
               Sets.newHashSet(testGroupName));
           fail("Exception should have been thrown");
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/thrift/TestSentryGenericServiceIntegration.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/thrift/TestSentryGenericServiceIntegration.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/thrift/TestSentryGenericServiceIntegration.java
index b59d172..ac8b2a7 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/thrift/TestSentryGenericServiceIntegration.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/thrift/TestSentryGenericServiceIntegration.java
@@ -54,7 +54,7 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
 
         client.createRole(requestorUserName, roleName, SOLR);
 
-        client.addRoleToGroups(requestorUserName, roleName, SOLR, Sets.newHashSet(requestorUserGroupNames));
+        client.grantRoleToGroups(requestorUserName, roleName, SOLR, Sets.newHashSet(requestorUserGroupNames));
 
         Set<TSentryRole> roles = client.listUserRoles(requestorUserName,SOLR);
         assertEquals("Incorrect number of roles", 1, roles.size());
@@ -82,7 +82,7 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
 
         client.createRole(requestorUserName, roleName, SOLR);
 
-        client.addRoleToGroups(requestorUserName, roleName, SOLR, Sets.newHashSet(testGroupName));
+        client.grantRoleToGroups(requestorUserName, roleName, SOLR, Sets.newHashSet(testGroupName));
 
         Set<TSentryRole> roles = client.listUserRoles(requestorUserName,SOLR);
         assertEquals("Incorrect number of roles", 1, roles.size());
@@ -94,7 +94,7 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
           }
         }
 
-        client.deleteRoleToGroups(requestorUserName, roleName, SOLR, Sets.newHashSet(testGroupName));
+        client.revokeRoleFromGroups(requestorUserName, roleName, SOLR, Sets.newHashSet(testGroupName));
         roles = client.listUserRoles(requestorUserName,SOLR);
         assertEquals("Incorrect number of roles", 0, roles.size());
 
@@ -159,11 +159,11 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
             SearchConstants.QUERY);
 
         client.grantPrivilege(requestorUserName, roleName1, SOLR, queryPrivilege);
-        Set<TSentryPrivilege> listPrivilegesByRoleName = client.listPrivilegesByRoleName(requestorUserName, roleName1, SOLR, "service1");
+        Set<TSentryPrivilege> listPrivilegesByRoleName = client.listAllPrivilegesByRoleName(requestorUserName, roleName1, SOLR, "service1");
         assertTrue("Privilege not assigned to role1 !!", listPrivilegesByRoleName.size() == 1);
 
         client.grantPrivilege(requestorUserName, roleName2, SOLR, queryPrivilege);
-        listPrivilegesByRoleName = client.listPrivilegesByRoleName(requestorUserName, roleName2, SOLR, "service1");
+        listPrivilegesByRoleName = client.listAllPrivilegesByRoleName(requestorUserName, roleName2, SOLR, "service1");
         assertTrue("Privilege not assigned to role2 !!", listPrivilegesByRoleName.size() == 1);
       }});
   }
@@ -183,7 +183,7 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
 
         client.dropRoleIfExists(requestorUserName, roleName, SOLR);
         client.createRole(requestorUserName, roleName, SOLR);
-        client.addRoleToGroups(requestorUserName, roleName, SOLR, Sets.newHashSet(groupName));
+        client.grantRoleToGroups(requestorUserName, roleName, SOLR, Sets.newHashSet(groupName));
 
         Set<TSentryRole> groupRoles = client.listRolesByGroupName(requestorUserName, groupName,SOLR);
         assertTrue(groupRoles.size() == 1);
@@ -223,11 +223,11 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
 
         client.grantPrivilege(requestorUserName, roleName, SOLR, updatePrivilege);
         client.grantPrivilege(requestorUserName, roleName, SOLR, queryPrivilege);
-        Set<TSentryPrivilege> privileges = client.listPrivilegesByRoleName(requestorUserName, roleName, SOLR, "service1");
+        Set<TSentryPrivilege> privileges = client.listAllPrivilegesByRoleName(requestorUserName, roleName, SOLR, "service1");
         assertTrue(privileges.size() == 2);
 
         client.revokePrivilege(requestorUserName, roleName, SOLR, updatePrivilege);
-        privileges = client.listPrivilegesByRoleName(requestorUserName, roleName, SOLR, "service1");
+        privileges = client.listAllPrivilegesByRoleName(requestorUserName, roleName, SOLR, "service1");
         assertTrue(privileges.size() == 1);
       }});
   }
@@ -250,7 +250,7 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
             SearchConstants.QUERY);
 
         client.grantPrivilege(requestorUserName, roleName, SOLR, queryPrivilege);
-        assertEquals(1, client.listPrivilegesByRoleName(requestorUserName, roleName, SOLR, "service1").size());
+        assertEquals(1, client.listAllPrivilegesByRoleName(requestorUserName, roleName, SOLR, "service1").size());
       }});
   }
 
@@ -297,8 +297,8 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
         client.grantPrivilege(adminUser, grantRole, SOLR, grantPrivilege);
         client.grantPrivilege(adminUser, noGrantRole, SOLR, noGrantPrivilege);
 
-        client.addRoleToGroups(adminUser, grantRole, SOLR, grantOptionGroup);
-        client.addRoleToGroups(adminUser, noGrantRole, SOLR, noGrantOptionGroup);
+        client.grantRoleToGroups(adminUser, grantRole, SOLR, grantOptionGroup);
+        client.grantRoleToGroups(adminUser, noGrantRole, SOLR, noGrantOptionGroup);
 
         try {
           client.grantPrivilege(grantOptionUser,testRole,SOLR, testPrivilege);
@@ -342,7 +342,7 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
 
 
         client.createRole(adminUser, testRole, SOLR);
-        client.addRoleToGroups(adminUser, testRole, SOLR, testGroup);
+        client.grantRoleToGroups(adminUser, testRole, SOLR, testGroup);
 
         TSentryPrivilege queryPrivilege = new TSentryPrivilege(SOLR, "service1",
             fromAuthorizable(Arrays.asList(new Collection("c1"), new Field("f1"))),
@@ -355,7 +355,7 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
         client.grantPrivilege(adminUser, testRole, SOLR, queryPrivilege);
         client.grantPrivilege(adminUser, testRole, SOLR, updatePrivilege);
 
-        assertEquals(2, client.listPrivilegesByRoleName(testUser, testRole, SOLR, "service1").size());
+        assertEquals(2, client.listAllPrivilegesByRoleName(testUser, testRole, SOLR, "service1").size());
 
         assertEquals(1, client.listPrivilegesByRoleName(testUser, testRole,
             SOLR, "service1", Arrays.asList(new Collection("c1"))).size());
@@ -401,7 +401,7 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
         writePolicyFile();
 
         client.createRole(adminUser, testRole, SOLR);
-        client.addRoleToGroups(adminUser, testRole, SOLR, adminGroup);
+        client.grantRoleToGroups(adminUser, testRole, SOLR, adminGroup);
 
         TSentryPrivilege queryPrivilege = new TSentryPrivilege(SOLR, "service1",
         fromAuthorizable(Arrays.asList(new Collection("c1"), new Field("f1"))),
@@ -414,34 +414,34 @@ public class TestSentryGenericServiceIntegration extends SentryGenericServiceInt
         client.grantPrivilege(adminUser, testRole, SOLR, queryPrivilege);
         client.grantPrivilege(adminUser, testRole, SOLR, updatePrivilege);
 
-        //test listPrivilegsbyAuthorizable without requested group and active role set.
-        assertEquals(1, client.listPrivilegsbyAuthorizable(SOLR, "service1", adminUser,
+        //test listPrivilegesbyAuthorizable without requested group and active role set.
+        assertEquals(1, client.listPrivilegesbyAuthorizable(SOLR, "service1", adminUser,
             Sets.newHashSet(new String("Collection=c1->Field=f1")), null, null).size());
 
-        //test listPrivilegsbyAuthorizable with requested group (testGroup)
-        Map<String, TSentryPrivilegeMap> privilegeMap = client.listPrivilegsbyAuthorizable(SOLR,
+        //test listPrivilegesbyAuthorizable with requested group (testGroup)
+        Map<String, TSentryPrivilegeMap> privilegeMap = client.listPrivilegesbyAuthorizable(SOLR,
             "service1", adminUser, Sets.newHashSet(new String("Collection=c1->Field=f1")), testGroup, null);
         TSentryPrivilegeMap actualMap = privilegeMap.get(new String("Collection=c1->Field=f1"));
         assertEquals(0, actualMap.getPrivilegeMap().size());
 
-        //test listPrivilegsbyAuthorizable with active role set.
+        //test listPrivilegesbyAuthorizable with active role set.
         ActiveRoleSet roleSet = ActiveRoleSet.ALL;
-        assertEquals(1, client.listPrivilegsbyAuthorizable(SOLR, "service1", adminUser,
+        assertEquals(1, client.listPrivilegesbyAuthorizable(SOLR, "service1", adminUser,
             Sets.newHashSet(new String("Collection=c1->Field=f1")), null, roleSet).size());
-        privilegeMap = client.listPrivilegsbyAuthorizable(SOLR,
+        privilegeMap = client.listPrivilegesbyAuthorizable(SOLR,
           "service1", adminUser, Sets.newHashSet(new String("Collection=c1->Field=f1")), null, roleSet);
         actualMap = privilegeMap.get(new String("Collection=c1->Field=f1"));
         assertEquals(1, actualMap.getPrivilegeMap().size());
 
-        privilegeMap = client.listPrivilegsbyAuthorizable(SOLR,
+        privilegeMap = client.listPrivilegesbyAuthorizable(SOLR,
             "service1", testUser, Sets.newHashSet(new String("Collection=c1->Field=f1")), null, roleSet);
         actualMap = privilegeMap.get(new String("Collection=c1->Field=f1"));
         assertEquals(0, actualMap.getPrivilegeMap().size());
 
         // grant tesRole to testGroup.
-        client.addRoleToGroups(adminUser, testRole, SOLR, testGroup);
+        client.grantRoleToGroups(adminUser, testRole, SOLR, testGroup);
 
-        privilegeMap = client.listPrivilegsbyAuthorizable(SOLR,
+        privilegeMap = client.listPrivilegesbyAuthorizable(SOLR,
             "service1", testUser, Sets.newHashSet(new String("Collection=c1")), null, roleSet);
         actualMap = privilegeMap.get(new String("Collection=c1"));
         assertEquals(1, actualMap.getPrivilegeMap().size());

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryConfigToolSolr.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryConfigToolSolr.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryConfigToolSolr.java
index 4b274fd..3685073 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryConfigToolSolr.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryConfigToolSolr.java
@@ -77,7 +77,7 @@ public class TestSentryConfigToolSolr extends SentryGenericServiceIntegrationBas
     Set<TSentryRole> tRoles = client.listAllRoles(requestorName, SOLR);
     for (TSentryRole tRole : tRoles) {
       String role = tRole.getRoleName();
-      Set<TSentryPrivilege> privileges = client.listPrivilegesByRoleName(
+      Set<TSentryPrivilege> privileges = client.listAllPrivilegesByRoleName(
           requestorName, role, SOLR, service);
       for (TSentryPrivilege privilege : privileges) {
         client.revokePrivilege(requestorName, role, SOLR, privilege);
@@ -135,7 +135,7 @@ public class TestSentryConfigToolSolr extends SentryGenericServiceIntegrationBas
         // check privileges
         GenericPrivilegeConverter convert = new GenericPrivilegeConverter(SOLR, service);
         for (String role : roles) {
-          Set<TSentryPrivilege> privileges = client.listPrivilegesByRoleName(
+          Set<TSentryPrivilege> privileges = client.listAllPrivilegesByRoleName(
               requestorName, role, SOLR, service);
           Set<String> expectedPrivileges = privilegeMapping.get(role);
           assertEquals("Privilege set size doesn't match for role: " + role,

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellKafka.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellKafka.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellKafka.java
index c744ee3..80bbcf1 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellKafka.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellKafka.java
@@ -265,7 +265,7 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
             privs[ i ], "-conf",
             confPath.getAbsolutePath(), "-t", "kafka" };
           SentryShellGeneric.main(args);
-          Set<TSentryPrivilege> privileges = client.listPrivilegesByRoleName(requestorName,
+          Set<TSentryPrivilege> privileges = client.listAllPrivilegesByRoleName(requestorName,
             TEST_ROLE_NAME_1, KAFKA, service);
           assertEquals("Incorrect number of privileges. Received privileges: " + Arrays.toString(privileges.toArray()), privs.length - (i + 1), privileges.size());
         }

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellSolr.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellSolr.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellSolr.java
index d857107..55831a4 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellSolr.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellSolr.java
@@ -263,7 +263,7 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
             privs[ i ], "-conf",
             confPath.getAbsolutePath(), "-t", "solr" };
           SentryShellGeneric.main(args);
-          Set<TSentryPrivilege> privileges = client.listPrivilegesByRoleName(requestorName,
+          Set<TSentryPrivilege> privileges = client.listAllPrivilegesByRoleName(requestorName,
             TEST_ROLE_NAME_1, SOLR, service);
           assertEquals("Incorrect number of privileges", privs.length - (i + 1), privileges.size());
         }

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellSqoop.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellSqoop.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellSqoop.java
index 1f49cce..7bafd8c 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellSqoop.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryShellSqoop.java
@@ -262,7 +262,7 @@ public class TestSentryShellSqoop extends SentryGenericServiceIntegrationBase {
             privs[ i ], "-conf",
             confPath.getAbsolutePath(), "-t", "sqoop" };
           SentryShellGeneric.main(args);
-          Set<TSentryPrivilege> privileges = client.listPrivilegesByRoleName(requestorName,
+          Set<TSentryPrivilege> privileges = client.listAllPrivilegesByRoleName(requestorName,
             TEST_ROLE_NAME_1, AuthorizationComponent.SQOOP, service);
           assertEquals("Incorrect number of privileges. Received privileges: " + Arrays.toString(privileges.toArray()), privs.length - (i + 1), privileges.size());
         }

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryPolicyServiceClient.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryPolicyServiceClient.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryPolicyServiceClient.java
index e464ef1..6b1e5df 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryPolicyServiceClient.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryPolicyServiceClient.java
@@ -51,11 +51,11 @@ public class TestSentryPolicyServiceClient extends SentryServiceIntegrationBase
 
         client.dropRoleIfExists(requestorUserName, roleName);
         client.createRole(requestorUserName, roleName);
-        client.listRoles(requestorUserName);
+        client.listAllRoles(requestorUserName);
         stopSentryService();
         server = SentryServiceFactory.create(conf);
         startSentryService();
-        client.listRoles(requestorUserName);
+        client.listAllRoles(requestorUserName);
         client.dropRole(requestorUserName, roleName);
       }
     });

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceClientPool.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceClientPool.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceClientPool.java
index dff7e1a..3e9e2cd 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceClientPool.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceClientPool.java
@@ -52,11 +52,11 @@ public class TestSentryServiceClientPool extends SentryServiceIntegrationBase {
 
         client.dropRoleIfExists(requestorUserName, roleName);
         client.createRole(requestorUserName, roleName);
-        client.listRoles(requestorUserName);
+        client.listAllRoles(requestorUserName);
         stopSentryService();
         server = SentryServiceFactory.create(conf);
         startSentryService();
-        client.listRoles(requestorUserName);
+        client.listAllRoles(requestorUserName);
         client.dropRole(requestorUserName, roleName);
       }
     });
@@ -85,7 +85,7 @@ public class TestSentryServiceClientPool extends SentryServiceIntegrationBase {
               @Override
               public Boolean run() throws Exception {
                 try {
-                  client.listRoles(ADMIN_USER);
+                  client.listAllRoles(ADMIN_USER);
                   return true;
                 } catch (SentryUserException sue) {
                   return false;

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceFailureCase.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceFailureCase.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceFailureCase.java
index 2f4e8f6..fc65382 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceFailureCase.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceFailureCase.java
@@ -59,7 +59,7 @@ public class TestSentryServiceFailureCase extends SentryServiceIntegrationBase {
     try {
       connectToSentryService();
       String requestorUserName = ADMIN_USER;
-      client.listRoles(requestorUserName);
+      client.listAllRoles(requestorUserName);
       Assert.fail("Failed to receive Exception");
     } catch(Exception e) {
       LOGGER.info("Excepted exception", e);

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
index a05521f..27bbc93 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
@@ -62,7 +62,7 @@ public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
 
         client.createRole(requestorUserName, roleName);
 
-        Set<TSentryRole> roles = client.listRoles(requestorUserName);
+        Set<TSentryRole> roles = client.listAllRoles(requestorUserName);
         assertEquals("Incorrect number of roles", 1, roles.size());
 
         for (TSentryRole role:roles) {
@@ -403,7 +403,7 @@ public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
         client.dropRoleIfExists(requestorUserName, roleName);
         client.createRole(requestorUserName, roleName);
 
-        Set<TSentryRole> roles = client.listRoles(requestorUserName);
+        Set<TSentryRole> roles = client.listAllRoles(requestorUserName);
         assertEquals("Incorrect number of roles", 1, roles.size());
 
         client.grantRoleToGroup(requestorUserName, groupName, roleName);
@@ -437,7 +437,7 @@ public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
         client.dropRoleIfExists(requestorUserName, roleName);
         client.createRole(requestorUserName, roleName);
 
-        Set<TSentryRole> roles = client.listRoles(requestorUserName);
+        Set<TSentryRole> roles = client.listAllRoles(requestorUserName);
         assertEquals("Incorrect number of roles", 1, roles.size());
 
         client.grantDatabasePrivilege(requestorUserName, roleName, server, db, AccessConstants.ALL);
@@ -466,7 +466,7 @@ public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
         // Creating associated role
         client.dropRoleIfExists(requestorUserName, roleName);
         client.createRole(requestorUserName, roleName);
-        Set<TSentryRole> roles = client.listRoles(requestorUserName);
+        Set<TSentryRole> roles = client.listAllRoles(requestorUserName);
         assertEquals("Incorrect number of roles", 1, roles.size());
 
         client.grantURIPrivilege(requestorUserName, roleName, server, uri);

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceWithInvalidMsgSize.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceWithInvalidMsgSize.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceWithInvalidMsgSize.java
index 072d294..1896586 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceWithInvalidMsgSize.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceWithInvalidMsgSize.java
@@ -56,7 +56,7 @@ public class TestSentryServiceWithInvalidMsgSize extends SentryServiceIntegratio
         boolean exceptionThrown = false;
         try {
           // client throws exception when message size is larger than the client's thrift max message size.
-          clientWithSmallMaxMsgSize.listRoles(ADMIN_USER);
+          clientWithSmallMaxMsgSize.listAllRoles(ADMIN_USER);
         } catch (SentryUserException e) {
           exceptionThrown = true;
           Assert.assertTrue(e.getMessage().contains("Thrift exception occurred"));
@@ -69,9 +69,9 @@ public class TestSentryServiceWithInvalidMsgSize extends SentryServiceIntegratio
 
         // client can still talk with sentry server when message size is smaller.
         client.dropRoleIfExists(ADMIN_USER, ROLE_NAME);
-        client.listRoles(ADMIN_USER);
+        client.listAllRoles(ADMIN_USER);
         client.createRole(ADMIN_USER, ROLE_NAME);
-        client.listRoles(ADMIN_USER);
+        client.listAllRoles(ADMIN_USER);
       }
     });
   }
@@ -97,7 +97,7 @@ public class TestSentryServiceWithInvalidMsgSize extends SentryServiceIntegratio
         writePolicyFile();
 
         // client can talk with server when message size is smaller.
-        client.listRoles(ADMIN_USER);
+        client.listAllRoles(ADMIN_USER);
         client.createRole(ADMIN_USER, ROLE_NAME);
 
         boolean exceptionThrown = false;
@@ -112,7 +112,7 @@ public class TestSentryServiceWithInvalidMsgSize extends SentryServiceIntegratio
         }
 
         // client can still talk with sentry server when message size is smaller.
-        Set<TSentryRole> roles = client.listRoles(ADMIN_USER);
+        Set<TSentryRole> roles = client.listAllRoles(ADMIN_USER);
         Assert.assertTrue(roles.size() == 1);
         Assert.assertEquals(ROLE_NAME, roles.iterator().next().getRoleName());
       }

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/tools/TestSentryShellHive.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/tools/TestSentryShellHive.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/tools/TestSentryShellHive.java
index 8335bcc..adfd102 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/tools/TestSentryShellHive.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/tools/TestSentryShellHive.java
@@ -106,7 +106,7 @@ public class TestSentryShellHive extends SentryServiceIntegrationBase {
         SentryShellHive.main(args);
 
         // validate the result
-        Set<TSentryRole> roles = client.listRoles(requestorName);
+        Set<TSentryRole> roles = client.listAllRoles(requestorName);
         assertEquals("Incorrect number of roles", 0, roles.size());
       }
     });

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
index 7ed303c..7d9b3ba 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
@@ -201,7 +201,7 @@ public abstract class SentryServiceIntegrationBase extends SentryMiniKdcTestcase
         @Override
         public void runTestAsSubject() throws Exception {
           if (client != null) {
-            Set<TSentryRole> tRoles = client.listRoles(ADMIN_USER);
+            Set<TSentryRole> tRoles = client.listAllRoles(ADMIN_USER);
             if (tRoles != null) {
               for (TSentryRole tRole : tRoles) {
                 client.dropRole(ADMIN_USER, tRole.getRoleName());

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestConcurrentClients.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestConcurrentClients.java b/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestConcurrentClients.java
index bf871f0..798ad96 100644
--- a/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestConcurrentClients.java
+++ b/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestConcurrentClients.java
@@ -318,7 +318,7 @@ public class TestConcurrentClients extends AbstractTestWithStaticConfiguration {
             while (Long.compare(elapsedTime, SENTRY_CLIENT_TEST_DURATION_MS) <= 0) {
               LOGGER.info("Test role " + test_role + " runs " + elapsedTime + " ms.");
               client.createRole(ADMIN1, test_role);
-              client.listRoles(ADMIN1);
+              client.listAllRoles(ADMIN1);
               client.grantServerPrivilege(ADMIN1, test_role, "server1", false);
               client.listAllPrivilegesByRoleName(ADMIN1, test_role);
               client.dropRole(ADMIN1, test_role);

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/metastore/SentryPolicyProviderForDb.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/metastore/SentryPolicyProviderForDb.java b/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/metastore/SentryPolicyProviderForDb.java
index b13165c..6739dd6 100644
--- a/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/metastore/SentryPolicyProviderForDb.java
+++ b/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/metastore/SentryPolicyProviderForDb.java
@@ -73,7 +73,7 @@ public class SentryPolicyProviderForDb extends PolicyFile {
     }
 
     // remove existing metadata
-    for (TSentryRole tRole : sentryClient.listRoles(StaticUserGroup.ADMIN1)) {
+    for (TSentryRole tRole : sentryClient.listAllRoles(StaticUserGroup.ADMIN1)) {
       sentryClient.dropRole(StaticUserGroup.ADMIN1, tRole.getRoleName());
     }
 

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestConcurrentClients.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestConcurrentClients.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestConcurrentClients.java
index 13b6e62..f8b6a15 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestConcurrentClients.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestConcurrentClients.java
@@ -340,7 +340,7 @@ public class TestConcurrentClients extends AbstractTestWithStaticConfiguration {
               client.grantRoleToGroup(ADMIN1, ADMINGROUP, test_role);
 
               // validate role
-              Set<TSentryRole> sentryRoles = client.listRoles(ADMIN1);
+              Set<TSentryRole> sentryRoles = client.listAllRoles(ADMIN1);
               String results = "";
               for (TSentryRole role : sentryRoles) {
                 results += role.toString() + "|";

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/SentryPolicyProviderForDb.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/SentryPolicyProviderForDb.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/SentryPolicyProviderForDb.java
index b13165c..6739dd6 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/SentryPolicyProviderForDb.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/SentryPolicyProviderForDb.java
@@ -73,7 +73,7 @@ public class SentryPolicyProviderForDb extends PolicyFile {
     }
 
     // remove existing metadata
-    for (TSentryRole tRole : sentryClient.listRoles(StaticUserGroup.ADMIN1)) {
+    for (TSentryRole tRole : sentryClient.listAllRoles(StaticUserGroup.ADMIN1)) {
       sentryClient.dropRole(StaticUserGroup.ADMIN1, tRole.getRoleName());
     }
 

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/AbstractKafkaSentryTestBase.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/AbstractKafkaSentryTestBase.java b/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/AbstractKafkaSentryTestBase.java
index 100d885..868d198 100644
--- a/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/AbstractKafkaSentryTestBase.java
+++ b/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/AbstractKafkaSentryTestBase.java
@@ -180,7 +180,7 @@ public class AbstractKafkaSentryTestBase {
     try (SentryGenericServiceClient sentryClient = getSentryClient()){
       // grant all privilege to admin user
       sentryClient.createRoleIfNotExist(ADMIN_USER, ADMIN_ROLE, COMPONENT);
-      sentryClient.addRoleToGroups(ADMIN_USER, ADMIN_ROLE, COMPONENT, Sets.newHashSet(ADMIN_GROUP));
+      sentryClient.grantRoleToGroups(ADMIN_USER, ADMIN_ROLE, COMPONENT, Sets.newHashSet(ADMIN_GROUP));
       final ArrayList<TAuthorizable> authorizables = new ArrayList<TAuthorizable>();
       Host host = new Host(InetAddress.getLocalHost().getHostName());
       authorizables.add(new TAuthorizable(host.getTypeName(), host.getName()));

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/TestAuthorize.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/TestAuthorize.java b/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/TestAuthorize.java
index 53f1f47..d6dfd7a 100644
--- a/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/TestAuthorize.java
+++ b/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/TestAuthorize.java
@@ -311,7 +311,7 @@ public class TestAuthorize extends AbstractKafkaSentryTestBase {
     SentryGenericServiceClient sentryClient = getSentryClient();
     try {
       sentryClient.createRoleIfNotExist(ADMIN_USER, role, COMPONENT);
-      sentryClient.addRoleToGroups(ADMIN_USER, role, COMPONENT, Sets.newHashSet(group));
+      sentryClient.grantRoleToGroups(ADMIN_USER, role, COMPONENT, Sets.newHashSet(group));
 
       sentryClient.grantPrivilege(ADMIN_USER, role, COMPONENT,
               new TSentryPrivilege(COMPONENT, "kafka", authorizables,

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/AbstractSolrSentryTestWithDbProvider.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/AbstractSolrSentryTestWithDbProvider.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/AbstractSolrSentryTestWithDbProvider.java
index b2aa9f4..71c3cb6 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/AbstractSolrSentryTestWithDbProvider.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/AbstractSolrSentryTestWithDbProvider.java
@@ -279,7 +279,7 @@ public class AbstractSolrSentryTestWithDbProvider extends AbstractSolrSentryTest
 
     for (int i = 0; i < roles.length; i++) {
       client.createRole(ADMIN_USER, roles[i], COMPONENT_SOLR);
-      client.addRoleToGroups(ADMIN_USER, roles[i], COMPONENT_SOLR, Sets.newHashSet(groups[i]));
+      client.grantRoleToGroups(ADMIN_USER, roles[i], COMPONENT_SOLR, Sets.newHashSet(groups[i]));
     }
 
     /**
@@ -287,7 +287,7 @@ public class AbstractSolrSentryTestWithDbProvider extends AbstractSolrSentryTest
      * grant ALL privilege on collection ALL to role admin
      */
     client.createRole(ADMIN_USER, ADMIN_ROLE, COMPONENT_SOLR);
-    client.addRoleToGroups(ADMIN_USER, ADMIN_ROLE, COMPONENT_SOLR, Sets.newHashSet(ADMIN_GROUP));
+    client.grantRoleToGroups(ADMIN_USER, ADMIN_ROLE, COMPONENT_SOLR, Sets.newHashSet(ADMIN_GROUP));
     grantCollectionPrivilege(SearchConstants.ALL, ADMIN_USER, ADMIN_ROLE, SearchConstants.ALL);
   }
 

http://git-wip-us.apache.org/repos/asf/sentry/blob/cfa8abb0/sentry-tests/sentry-tests-sqoop/src/test/java/org/apache/sentry/tests/e2e/sqoop/AbstractSqoopSentryTestBase.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-sqoop/src/test/java/org/apache/sentry/tests/e2e/sqoop/AbstractSqoopSentryTestBase.java b/sentry-tests/sentry-tests-sqoop/src/test/java/org/apache/sentry/tests/e2e/sqoop/AbstractSqoopSentryTestBase.java
index b27dbc2..2971bf8 100644
--- a/sentry-tests/sentry-tests-sqoop/src/test/java/org/apache/sentry/tests/e2e/sqoop/AbstractSqoopSentryTestBase.java
+++ b/sentry-tests/sentry-tests-sqoop/src/test/java/org/apache/sentry/tests/e2e/sqoop/AbstractSqoopSentryTestBase.java
@@ -201,7 +201,7 @@ public class AbstractSqoopSentryTestBase {
                  SentryGenericServiceClientFactory.create(getClientConfig())){
       // grant all privilege to admin user
       sentryClient.createRoleIfNotExist(ADMIN_USER, ADMIN_ROLE, COMPONENT);
-      sentryClient.addRoleToGroups(ADMIN_USER, ADMIN_ROLE, COMPONENT, Sets.newHashSet(ADMIN_GROUP));
+      sentryClient.grantRoleToGroups(ADMIN_USER, ADMIN_ROLE, COMPONENT, Sets.newHashSet(ADMIN_GROUP));
       sentryClient.grantPrivilege(ADMIN_USER, ADMIN_ROLE, COMPONENT,
           new TSentryPrivilege(COMPONENT, SQOOP_SERVER_NAME, new ArrayList<TAuthorizable>(),
               SqoopActionConstant.ALL));


[6/6] sentry git commit: Merge branch 'master' into akolb-cli

Posted by co...@apache.org.
Merge branch 'master' into akolb-cli


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

Branch: refs/heads/akolb-cli
Commit: 8be62797b1e6e476f1d012eb6a35feb128a708c2
Parents: 99f03c3 6fa0288
Author: Colm O hEigeartaigh <co...@apache.org>
Authored: Fri Oct 27 10:22:15 2017 +0100
Committer: Colm O hEigeartaigh <co...@apache.org>
Committed: Fri Oct 27 10:22:15 2017 +0100

----------------------------------------------------------------------
 .../DefaultSentryAccessController.java          |   2 +-
 .../hive/ql/exec/SentryGrantRevokeTask.java     |   2 +-
 .../authz/DefaultSentryAccessController.java    |   2 +-
 .../sentry/kafka/binding/KafkaAuthBinding.java  |   6 +-
 .../sentry/sqoop/binding/SqoopAuthBinding.java  |   8 +-
 .../sentry/core/common/utils/PathUtils.java     |   9 ++
 .../org/apache/sentry/hdfs/PathsUpdate.java     |   9 +-
 .../org/apache/sentry/hdfs/TestPathsUpdate.java |  32 ++++--
 .../provider/db/generic/UpdatableCache.java     |   2 +-
 .../thrift/SentryGenericServiceClient.java      |  14 +--
 .../SentryGenericServiceClientDefaultImpl.java  |  12 +-
 .../tools/GenericPrivilegeConverter.java        |  13 ++-
 .../db/generic/tools/SentryConfigToolSolr.java  |   2 +-
 .../db/generic/tools/SentryShellGeneric.java    |  58 +++++-----
 .../tools/command/AddRoleToGroupCmd.java        |  46 --------
 .../db/generic/tools/command/Command.java       |  27 -----
 .../db/generic/tools/command/CreateRoleCmd.java |  39 -------
 .../tools/command/DeleteRoleFromGroupCmd.java   |  46 --------
 .../db/generic/tools/command/DropRoleCmd.java   |  39 -------
 .../tools/command/GenericShellCommand.java      | 112 +++++++++++++++++++
 .../tools/command/GrantPrivilegeToRoleCmd.java  |  47 --------
 .../tools/command/ListPrivilegesByRoleCmd.java  |  54 ---------
 .../db/generic/tools/command/ListRolesCmd.java  |  53 ---------
 .../command/RevokePrivilegeFromRoleCmd.java     |  47 --------
 .../command/TSentryPrivilegeConverter.java      |   3 +-
 .../db/service/persistent/SentryStore.java      |   3 +-
 .../thrift/SentryPolicyServiceClient.java       |   2 +-
 .../SentryPolicyServiceClientDefaultImpl.java   |   2 +-
 .../provider/db/tools/SentryShellHive.java      |  42 +++----
 .../sentry/provider/db/tools/ShellCommand.java  |  44 ++++++++
 .../provider/db/tools/command/hive/Command.java |  27 -----
 .../db/tools/command/hive/CommandUtil.java      |   2 +-
 .../db/tools/command/hive/CreateRoleCmd.java    |  37 ------
 .../db/tools/command/hive/DropRoleCmd.java      |  37 ------
 .../command/hive/GrantPrivilegeToRoleCmd.java   |  43 -------
 .../command/hive/GrantRoleToGroupsCmd.java      |  44 --------
 .../db/tools/command/hive/HiveShellCommand.java | 108 ++++++++++++++++++
 .../tools/command/hive/ListPrivilegesCmd.java   |  49 --------
 .../db/tools/command/hive/ListRolesCmd.java     |  51 ---------
 .../hive/RevokePrivilegeFromRoleCmd.java        |  44 --------
 .../command/hive/RevokeRoleFromGroupsCmd.java   |  43 -------
 .../service/thrift/NotificationProcessor.java   |   3 +-
 .../TestAuditLogForSentryGenericService.java    |   8 +-
 .../TestSentryGenericServiceIntegration.java    |  48 ++++----
 .../generic/tools/TestSentryConfigToolSolr.java |   4 +-
 .../db/generic/tools/TestSentryShellKafka.java  |   2 +-
 .../db/generic/tools/TestSentryShellSolr.java   |   2 +-
 .../db/generic/tools/TestSentryShellSqoop.java  |   2 +-
 .../thrift/TestSentryPolicyServiceClient.java   |   4 +-
 .../thrift/TestSentryServiceClientPool.java     |   6 +-
 .../thrift/TestSentryServiceFailureCase.java    |   2 +-
 .../thrift/TestSentryServiceIntegration.java    |   8 +-
 .../TestSentryServiceWithInvalidMsgSize.java    |  10 +-
 .../provider/db/tools/TestSentryShellHive.java  |   2 +-
 .../thrift/SentryServiceIntegrationBase.java    |   2 +-
 .../e2e/dbprovider/TestConcurrentClients.java   |   2 +-
 .../metastore/SentryPolicyProviderForDb.java    |   2 +-
 .../e2e/dbprovider/TestConcurrentClients.java   |   2 +-
 .../AbstractTestWithStaticConfiguration.java    |   7 +-
 .../metastore/SentryPolicyProviderForDb.java    |   2 +-
 .../e2e/kafka/AbstractKafkaSentryTestBase.java  |  14 ++-
 .../sentry/tests/e2e/kafka/TestAuthorize.java   |   5 +-
 .../AbstractSolrSentryTestWithDbProvider.java   |   4 +-
 .../e2e/sqoop/AbstractSqoopSentryTestBase.java  |   2 +-
 .../java/org/apache/sentry/shell/ShellUtil.java |   6 +-
 65 files changed, 472 insertions(+), 939 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/8be62797/sentry-tools/src/main/java/org/apache/sentry/shell/ShellUtil.java
----------------------------------------------------------------------
diff --cc sentry-tools/src/main/java/org/apache/sentry/shell/ShellUtil.java
index bea53c8,0000000..daf9b73
mode 100644,000000..100644
--- a/sentry-tools/src/main/java/org/apache/sentry/shell/ShellUtil.java
+++ b/sentry-tools/src/main/java/org/apache/sentry/shell/ShellUtil.java
@@@ -1,261 -1,0 +1,261 @@@
 +/*
 + * 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.sentry.shell;
 +
 +import com.google.common.collect.Sets;
 +import org.apache.commons.lang.StringUtils;
 +import org.apache.sentry.core.common.exception.SentryUserException;
 +import org.apache.sentry.provider.db.service.thrift.*;
 +import org.apache.sentry.provider.db.tools.command.hive.CommandUtil;
 +
 +import java.util.*;
 +
 +import static org.apache.sentry.service.thrift.SentryServiceUtil.convertTSentryPrivilegeToStr;
 +import static org.apache.sentry.service.thrift.SentryServiceUtil.convertToTSentryPrivilege;
 +
 +/**
 + * ShellUtil implements actual commands
 + */
 +class ShellUtil {
 +
 +    private final SentryPolicyServiceClient sentryClient;
 +    private final String authUser;
 +
 +    ShellUtil(SentryPolicyServiceClient sentryClient, String authUser) {
 +        this.sentryClient = sentryClient;
 +        this.authUser = authUser;
 +    }
 +
 +    List<String> listRoles() {
 +        return listRoles(null);
 +    }
 +
 +    List<String> listRoles(String group) {
 +        Set<TSentryRole> roles = null;
 +        try {
 +            if (StringUtils.isEmpty(group)) {
-                 roles = sentryClient.listRoles(authUser);
++                roles = sentryClient.listAllRoles(authUser);
 +            } else {
 +                roles = sentryClient.listRolesByGroupName(authUser, group);
 +            }
 +        } catch (SentryUserException e) {
 +            System.out.println("Error listing roles: " + e.toString());
 +        }
 +        List<String> result = new ArrayList<>();
 +        if (roles == null || roles.isEmpty()) {
 +            return result;
 +        }
 +
 +        for (TSentryRole role : roles) {
 +            result.add(role.getRoleName());
 +        }
 +
 +        Collections.sort(result);
 +        return result;
 +    }
 +
 +    void createRoles(String ...roles) {
 +        for (String role : roles) {
 +            try {
 +                sentryClient.createRole(authUser, role);
 +            } catch (SentryUserException e) {
 +                System.out.printf("failed to create role %s: %s\n",
 +                        role, e.toString());
 +            }
 +        }
 +    }
 +
 +    void dropRoles(String ...roles) {
 +        for (String role : roles) {
 +            try {
 +                sentryClient.dropRole(authUser, role);
 +            } catch (SentryUserException e) {
 +                System.out.printf("failed to drop role %s: %s\n",
 +                        role, e.toString());
 +            }
 +        }
 +    }
 +
 +    List<String> listGroups() {
 +        Set<TSentryRole> roles = null;
 +
 +        try {
-             roles = sentryClient.listRoles(authUser);
++            roles = sentryClient.listAllRoles(authUser);
 +        } catch (SentryUserException e) {
 +            System.out.println("Error reading roles: " + e.toString());
 +        }
 +
 +        if (roles == null || roles.isEmpty()) {
 +            return new ArrayList<>();
 +        }
 +
 +        // Set of all group names
 +        Set<String> groupNames = new HashSet<>();
 +
 +        // Get all group names
 +        for (TSentryRole role: roles) {
 +            for (TSentryGroup group: role.getGroups()) {
 +                groupNames.add(group.getGroupName());
 +            }
 +        }
 +
 +        List<String> result = new ArrayList<>(groupNames);
 +
 +        Collections.sort(result);
 +        return result;
 +    }
 +
 +    List<String> listGroupRoles() {
 +        Set<TSentryRole> roles = null;
 +
 +        try {
-             roles = sentryClient.listRoles(authUser);
++            roles = sentryClient.listAllRoles(authUser);
 +        } catch (SentryUserException e) {
 +            System.out.println("Error reading roles: " + e.toString());
 +        }
 +
 +        if (roles == null || roles.isEmpty()) {
 +            return new ArrayList<>();
 +        }
 +
 +        // Set of all group names
 +        Set<String> groupNames = new HashSet<>();
 +
 +        // Map group to set of roles
 +        Map<String, Set<String>> groupInfo = new HashMap<>();
 +
 +        // Get all group names
 +        for (TSentryRole role: roles) {
 +            for (TSentryGroup group: role.getGroups()) {
 +                String groupName = group.getGroupName();
 +                groupNames.add(groupName);
 +                Set<String> groupRoles = groupInfo.get(groupName);
 +                if (groupRoles != null) {
 +                    // Add a new or existing role
 +                    groupRoles.add(role.getRoleName());
 +                    continue;
 +                }
 +                // Never seen this group before
 +                groupRoles = new HashSet<>();
 +                groupRoles.add(role.getRoleName());
 +                groupInfo.put(groupName, groupRoles);
 +            }
 +        }
 +
 +        List<String> groups = new ArrayList<>(groupNames);
 +        Collections.sort(groups);
 +
 +        // Produce printable result as
 +        // group1 = role1, role2, ...
 +        // group2 = ...
 +        List<String> result = new LinkedList<>();
 +        for(String groupName: groups) {
 +            result.add(groupName + " = " +
 +                    StringUtils.join(groupInfo.get(groupName), ", "));
 +        }
 +        return result;
 +    }
 +
 +    void grantGroupsToRole(String roleName, String ...groups) {
 +        try {
 +            sentryClient.grantRoleToGroups(authUser, roleName, Sets.newHashSet(groups));
 +        } catch (SentryUserException e) {
 +            System.out.printf("Failed to gran role %s to groups: %s\n",
 +                    roleName, e.toString());
 +        }
 +    }
 +
 +    void revokeGroupsFromRole(String roleName, String ...groups) {
 +        try {
 +            sentryClient.revokeRoleFromGroups(authUser, roleName, Sets.newHashSet(groups));
 +        } catch (SentryUserException e) {
 +            System.out.printf("Failed to revoke role %s to groups: %s\n",
 +                    roleName, e.toString());
 +        }
 +    }
 +
 +    void grantPrivilegeToRole(String roleName, String privilege) {
 +        TSentryPrivilege tPriv = convertToTSentryPrivilege(privilege);
 +        try {
 +            CommandUtil.validatePrivilegeHierarchy(tPriv);
 +            sentryClient.grantPrivilege(authUser, roleName, tPriv);
 +        } catch (SentryUserException | IllegalArgumentException e) {
 +            System.out.println("Error granting privilege: " + e.toString());
 +        }
 +    }
 +
 +    List<String> listPrivileges(String roleName) {
 +        Set<TSentryPrivilege> privileges = null;
 +        try {
 +            privileges = sentryClient
 +                    .listAllPrivilegesByRoleName(authUser, roleName);
 +        } catch (SentryUserException e) {
 +            System.out.println("Failed to list privileges: " + e.toString());
 +        }
 +
 +        List<String> result = new LinkedList<>();
 +        if (privileges == null || privileges.isEmpty()) {
 +            return result;
 +        }
 +
 +        for (TSentryPrivilege privilege : privileges) {
 +            String privilegeStr = convertTSentryPrivilegeToStr(privilege);
 +            if (privilegeStr.isEmpty()) {
 +                continue;
 +            }
 +            result.add(privilegeStr);
 +        }
 +        return result;
 +    }
 +
 +    /**
 +     * List all privileges
 +     * @return string with privilege info for all roles
 +     */
 +    String listPrivileges() {
 +        List<String> roles = listRoles(null);
 +        if (roles == null || roles.isEmpty()) {
 +            return "";
 +        }
 +
 +        StringBuilder result = new StringBuilder();
 +        for (String role: roles) {
 +            List<String> privs = listPrivileges(role);
 +            if (privs.isEmpty()) {
 +                continue;
 +            }
 +            result.append(role).append(" = ");
 +            result.append(StringUtils.join(listPrivileges(role), ",\n\t"));
 +            result.append('\n');
 +        }
 +        return result.toString();
 +    }
 +
 +    void revokePrivilegeFromRole(String roleName, String privilegeStr) {
 +        TSentryPrivilege tSentryPrivilege = convertToTSentryPrivilege(privilegeStr);
 +        try {
 +            CommandUtil.validatePrivilegeHierarchy(tSentryPrivilege);
 +            sentryClient.revokePrivilege(authUser, roleName, tSentryPrivilege);
 +        } catch (SentryUserException | IllegalArgumentException e) {
 +            System.out.println("failed to revoke privilege: " + e.toString());
 +        }
 +    }
 +
 +
 +}


[4/6] sentry git commit: SENTRY-2017: Fix Sentry e2e tests to use SentryMetastorePostEventListenerNotificationLog.

Posted by co...@apache.org.
SENTRY-2017: Fix Sentry e2e tests to use SentryMetastorePostEventListenerNotificationLog.


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

Branch: refs/heads/akolb-cli
Commit: 99efc7fc960d5680fc7605400e0375a7883bae37
Parents: cfa8abb
Author: Kalyan Kumar Kalvagadda <kk...@cloudera.com>
Authored: Mon Oct 23 15:37:10 2017 -0500
Committer: Kalyan Kumar Kalvagadda <kk...@cloudera.com>
Committed: Thu Oct 26 13:56:29 2017 -0500

----------------------------------------------------------------------
 .../tests/e2e/hive/AbstractTestWithStaticConfiguration.java   | 7 +------
 1 file changed, 1 insertion(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/99efc7fc/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
index 5fb4659..630bc68 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
@@ -128,7 +128,6 @@ public abstract class AbstractTestWithStaticConfiguration extends RulesForE2ETes
   protected static final String SERVER_HOST = "localhost";
   private static final String EXTERNAL_SENTRY_SERVICE = "sentry.e2etest.external.sentry";
   protected static final String EXTERNAL_HIVE_LIB = "sentry.e2etest.hive.lib";
-  private static final String ENABLE_NOTIFICATION_LOG = "sentry.e2etest.enable.notification.log";
 
   protected static boolean policyOnHdfs = false;
   protected static boolean defaultFSOnHdfs = false;
@@ -152,7 +151,7 @@ public abstract class AbstractTestWithStaticConfiguration extends RulesForE2ETes
   protected static Map<String, String> properties;
   protected static SentrySrv sentryServer;
   protected static Configuration sentryConf;
-  protected static boolean enableNotificationLog = false;
+  protected static boolean enableNotificationLog = true;
   protected static Context context;
   protected final String semanticException = "SemanticException No valid privileges";
 
@@ -282,10 +281,6 @@ public abstract class AbstractTestWithStaticConfiguration extends RulesForE2ETes
       policyURI = policyFileLocation.getPath();
     }
 
-    if ("true".equalsIgnoreCase(System.getProperty(ENABLE_NOTIFICATION_LOG, "false"))) {
-      enableNotificationLog = true;
-    }
-
     if (enableHiveConcurrency) {
       properties.put(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "true");
       properties.put(HiveConf.ConfVars.HIVE_TXN_MANAGER.varname,


[5/6] sentry git commit: SENTRY-2020: Fix testConsumeCycleWithInsufficientPrivileges test failure in kafka e2e tests.

Posted by co...@apache.org.
SENTRY-2020: Fix testConsumeCycleWithInsufficientPrivileges test failure in kafka e2e tests.


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

Branch: refs/heads/akolb-cli
Commit: 6fa02888fc6b5bc866f9ece344e03ce95e9ebf86
Parents: 99efc7f
Author: Kalyan Kumar Kalvagadda <kk...@cloudera.com>
Authored: Thu Oct 26 12:25:23 2017 -0500
Committer: Kalyan Kumar Kalvagadda <kk...@cloudera.com>
Committed: Thu Oct 26 14:05:39 2017 -0500

----------------------------------------------------------------------
 .../tests/e2e/kafka/AbstractKafkaSentryTestBase.java    | 12 ++++++++++++
 .../apache/sentry/tests/e2e/kafka/TestAuthorize.java    |  3 ++-
 2 files changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/6fa02888/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/AbstractKafkaSentryTestBase.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/AbstractKafkaSentryTestBase.java b/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/AbstractKafkaSentryTestBase.java
index 868d198..33ef8c4 100644
--- a/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/AbstractKafkaSentryTestBase.java
+++ b/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/AbstractKafkaSentryTestBase.java
@@ -204,6 +204,18 @@ public class AbstractKafkaSentryTestBase {
     }
   }
 
+  public static void assertCausedMessages(Exception e, String message1, String message2) {
+    if (e.getCause() != null) {
+      assertTrue("Expected message: " + message1 + " OR " + message2 ,
+              (e.getCause().getMessage().contains(message1) ||
+                      e.getCause().getMessage().contains(message2)));
+    } else {
+      assertTrue("Expected message: " + message1 + " OR " + message2 + ", but got: " + e.getMessage(),
+              (e.getMessage().contains(message1) ||
+                      e.getMessage().contains(message2)));
+    }
+  }
+
   private static Configuration getClientConfig() {
     Configuration conf = new Configuration();
     /** set the Sentry client configuration for Kafka Service integration */

http://git-wip-us.apache.org/repos/asf/sentry/blob/6fa02888/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/TestAuthorize.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/TestAuthorize.java b/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/TestAuthorize.java
index d6dfd7a..cc19f64 100644
--- a/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/TestAuthorize.java
+++ b/sentry-tests/sentry-tests-kafka/src/test/java/org/apache/sentry/tests/e2e/kafka/TestAuthorize.java
@@ -281,7 +281,8 @@ public class TestAuthorize extends AbstractKafkaSentryTestBase {
       testConsume(TOPIC_NAME, StaticUserGroupRole.USER_1);
       Assert.fail("user1 must not have been authorized to read consumer group sentrykafkaconsumer.");
     } catch (Exception ex) {
-      assertCausedMessage(ex, "Not authorized to access topics: [" + TOPIC_NAME + "]");
+      assertCausedMessages(ex, "Not authorized to access group: sentrykafkaconsumer",
+              "Not authorized to access topics: [" + TOPIC_NAME + "]");
     }
 
   /*


[2/6] sentry git commit: SENTRY-2015 - Refactor Command implementations - Reviewed by Sergio Pena

Posted by co...@apache.org.
SENTRY-2015 - Refactor Command implementations
 - Reviewed by Sergio Pena


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

Branch: refs/heads/akolb-cli
Commit: 6d44cfd9b5c8ec12fdf5310bb86cfa09e7b84562
Parents: 7dbadfe
Author: Colm O hEigeartaigh <co...@apache.org>
Authored: Wed Oct 25 09:27:44 2017 +0100
Committer: Colm O hEigeartaigh <co...@apache.org>
Committed: Wed Oct 25 09:27:44 2017 +0100

----------------------------------------------------------------------
 .../tools/GenericPrivilegeConverter.java        |  13 ++-
 .../db/generic/tools/SentryShellGeneric.java    |  58 +++++-----
 .../tools/command/AddRoleToGroupCmd.java        |  46 --------
 .../db/generic/tools/command/Command.java       |  27 -----
 .../db/generic/tools/command/CreateRoleCmd.java |  39 -------
 .../tools/command/DeleteRoleFromGroupCmd.java   |  46 --------
 .../db/generic/tools/command/DropRoleCmd.java   |  39 -------
 .../tools/command/GenericShellCommand.java      | 112 +++++++++++++++++++
 .../tools/command/GrantPrivilegeToRoleCmd.java  |  47 --------
 .../tools/command/ListPrivilegesByRoleCmd.java  |  54 ---------
 .../db/generic/tools/command/ListRolesCmd.java  |  53 ---------
 .../command/RevokePrivilegeFromRoleCmd.java     |  47 --------
 .../command/TSentryPrivilegeConverter.java      |   3 +-
 .../provider/db/tools/SentryShellHive.java      |  42 +++----
 .../sentry/provider/db/tools/ShellCommand.java  |  44 ++++++++
 .../provider/db/tools/command/hive/Command.java |  27 -----
 .../db/tools/command/hive/CommandUtil.java      |   2 +-
 .../db/tools/command/hive/CreateRoleCmd.java    |  37 ------
 .../db/tools/command/hive/DropRoleCmd.java      |  37 ------
 .../command/hive/GrantPrivilegeToRoleCmd.java   |  43 -------
 .../command/hive/GrantRoleToGroupsCmd.java      |  44 --------
 .../db/tools/command/hive/HiveShellCommand.java | 108 ++++++++++++++++++
 .../tools/command/hive/ListPrivilegesCmd.java   |  49 --------
 .../db/tools/command/hive/ListRolesCmd.java     |  51 ---------
 .../hive/RevokePrivilegeFromRoleCmd.java        |  44 --------
 .../command/hive/RevokeRoleFromGroupsCmd.java   |  43 -------
 26 files changed, 324 insertions(+), 831 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/GenericPrivilegeConverter.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/GenericPrivilegeConverter.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/GenericPrivilegeConverter.java
index 526a521..51d6df9 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/GenericPrivilegeConverter.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/GenericPrivilegeConverter.java
@@ -29,6 +29,7 @@ import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.sentry.core.common.Authorizable;
+import org.apache.sentry.core.common.exception.SentryUserException;
 import org.apache.sentry.core.common.utils.KeyValue;
 import org.apache.sentry.core.common.utils.PolicyFileConstants;
 import org.apache.sentry.core.common.utils.SentryConstants;
@@ -70,7 +71,7 @@ public class GenericPrivilegeConverter implements TSentryPrivilegeConverter {
     this.validate = validate;
   }
 
-  public TSentryPrivilege fromString(String privilegeStr) throws Exception {
+  public TSentryPrivilege fromString(String privilegeStr) throws SentryUserException {
     privilegeStr = parsePrivilegeString(privilegeStr);
     if (validate) {
       validatePrivilegeHierarchy(privilegeStr);
@@ -145,7 +146,7 @@ public class GenericPrivilegeConverter implements TSentryPrivilegeConverter {
     return privilegeStr;
   }
 
-  private void validatePrivilegeHierarchy(String privilegeStr) throws Exception {
+  private void validatePrivilegeHierarchy(String privilegeStr) throws SentryUserException {
     List<PrivilegeValidator> validators = getPrivilegeValidators();
     PrivilegeValidatorContext context = new PrivilegeValidatorContext(null, privilegeStr);
     for (PrivilegeValidator validator : validators) {
@@ -157,7 +158,7 @@ public class GenericPrivilegeConverter implements TSentryPrivilegeConverter {
     }
   }
 
-  private List<PrivilegeValidator> getPrivilegeValidators() throws Exception {
+  private List<PrivilegeValidator> getPrivilegeValidators() throws SentryUserException {
     if (AuthorizationComponent.KAFKA.equals(component)) {
       return KafkaPrivilegeModel.getInstance().getPrivilegeValidators();
     } else if ("SOLR".equals(component)) {
@@ -166,10 +167,10 @@ public class GenericPrivilegeConverter implements TSentryPrivilegeConverter {
       return SqoopPrivilegeModel.getInstance().getPrivilegeValidators(service);
     }
 
-    throw new Exception("Invalid component specified for GenericPrivilegeCoverter: " + component);
+    throw new SentryUserException("Invalid component specified for GenericPrivilegeCoverter: " + component);
   }
 
-  private Authorizable getAuthorizable(KeyValue keyValue) throws Exception {
+  private Authorizable getAuthorizable(KeyValue keyValue) throws SentryUserException {
     if (AuthorizationComponent.KAFKA.equals(component)) {
       return KafkaModelAuthorizables.from(keyValue);
     } else if ("SOLR".equals(component)) {
@@ -178,7 +179,7 @@ public class GenericPrivilegeConverter implements TSentryPrivilegeConverter {
       return SqoopModelAuthorizables.from(keyValue);
     }
 
-    throw new Exception("Invalid component specified for GenericPrivilegeCoverter: " + component);
+    throw new SentryUserException("Invalid component specified for GenericPrivilegeCoverter: " + component);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellGeneric.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellGeneric.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellGeneric.java
index 25c8003..49f18c8 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellGeneric.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellGeneric.java
@@ -18,6 +18,8 @@
 
 package org.apache.sentry.provider.db.generic.tools;
 
+import java.util.List;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -25,16 +27,10 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.sentry.provider.common.AuthorizationComponent;
 import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
 import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClientFactory;
-import org.apache.sentry.provider.db.generic.tools.command.AddRoleToGroupCmd;
-import org.apache.sentry.provider.db.generic.tools.command.Command;
-import org.apache.sentry.provider.db.generic.tools.command.CreateRoleCmd;
-import org.apache.sentry.provider.db.generic.tools.command.DeleteRoleFromGroupCmd;
-import org.apache.sentry.provider.db.generic.tools.command.DropRoleCmd;
-import org.apache.sentry.provider.db.generic.tools.command.GrantPrivilegeToRoleCmd;
-import org.apache.sentry.provider.db.generic.tools.command.ListPrivilegesByRoleCmd;
-import org.apache.sentry.provider.db.generic.tools.command.ListRolesCmd;
-import org.apache.sentry.provider.db.generic.tools.command.RevokePrivilegeFromRoleCmd;
+import org.apache.sentry.provider.db.generic.tools.command.GenericShellCommand;
+import org.apache.sentry.provider.db.generic.tools.command.TSentryPrivilegeConverter;
 import org.apache.sentry.provider.db.tools.SentryShellCommon;
+import org.apache.sentry.provider.db.tools.ShellCommand;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,7 +49,6 @@ public class SentryShellGeneric extends SentryShellCommon {
 
   @Override
   public void run() throws Exception {
-    Command command = null;
     String component = getComponent();
     Configuration conf = getSentryConf();
 
@@ -62,36 +57,37 @@ public class SentryShellGeneric extends SentryShellCommon {
                 SentryGenericServiceClientFactory.create(conf)) {
       UserGroupInformation ugi = UserGroupInformation.getLoginUser();
       String requestorName = ugi.getShortUserName();
+      TSentryPrivilegeConverter converter = new GenericPrivilegeConverter(component, service);
+      ShellCommand command = new GenericShellCommand(client, component, service, converter);
+
+      // check the requestor name
+      if (StringUtils.isEmpty(requestorName)) {
+        // The exception message will be recorded in log file.
+        throw new Exception("The requestor name is empty.");
+      }
 
       if (isCreateRole) {
-        command = new CreateRoleCmd(roleName, component);
+        command.createRole(requestorName, roleName);
       } else if (isDropRole) {
-        command = new DropRoleCmd(roleName, component);
+        command.dropRole(requestorName, roleName);
       } else if (isAddRoleGroup) {
-        command = new AddRoleToGroupCmd(roleName, groupName, component);
+        command.grantRoleToGroups(requestorName, roleName, groupName);
       } else if (isDeleteRoleGroup) {
-        command = new DeleteRoleFromGroupCmd(roleName, groupName, component);
+        command.revokeRoleFromGroups(requestorName, roleName, groupName);
       } else if (isGrantPrivilegeRole) {
-        command = new GrantPrivilegeToRoleCmd(roleName, component,
-                privilegeStr, new GenericPrivilegeConverter(component, service));
+        command.grantPrivilegeToRole(requestorName, roleName, privilegeStr);
       } else if (isRevokePrivilegeRole) {
-        command = new RevokePrivilegeFromRoleCmd(roleName, component,
-                privilegeStr, new GenericPrivilegeConverter(component, service));
+        command.revokePrivilegeFromRole(requestorName, roleName, privilegeStr);
       } else if (isListRole) {
-        command = new ListRolesCmd(groupName, component);
+        List<String> roles = command.listRoles(requestorName, roleName, groupName);
+        for (String role : roles) {
+          System.out.println(role);
+        }
       } else if (isListPrivilege) {
-        command = new ListPrivilegesByRoleCmd(roleName, component,
-                service, new GenericPrivilegeConverter(component, service));
-      }
-
-      // check the requestor name
-      if (StringUtils.isEmpty(requestorName)) {
-        // The exception message will be recorded in log file.
-        throw new Exception("The requestor name is empty.");
-      }
-
-      if (command != null) {
-        command.execute(client, requestorName);
+        List<String> privileges = command.listPrivileges(requestorName, roleName);
+        for (String privilege : privileges) {
+          System.out.println(privilege);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/AddRoleToGroupCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/AddRoleToGroupCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/AddRoleToGroupCmd.java
deleted file mode 100644
index a45d7e4..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/AddRoleToGroupCmd.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * 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.sentry.provider.db.generic.tools.command;
-
-import com.google.common.collect.Sets;
-import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
-import org.apache.sentry.provider.db.tools.SentryShellCommon;
-
-import java.util.Set;
-
-/**
- * Command for adding groups to a role.
- */
-public class AddRoleToGroupCmd implements Command {
-
-  private String roleName;
-  private String groups;
-  private String component;
-
-  public AddRoleToGroupCmd(String roleName, String groups, String component) {
-    this.roleName = roleName;
-    this.groups = groups;
-    this.component = component;
-  }
-
-  @Override
-  public void execute(SentryGenericServiceClient client, String requestorName) throws Exception {
-    Set<String> groupSet = Sets.newHashSet(groups.split(SentryShellCommon.GROUP_SPLIT_CHAR));
-    client.addRoleToGroups(requestorName, roleName, component, groupSet);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/Command.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/Command.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/Command.java
deleted file mode 100644
index e824fb3..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/Command.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * 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.sentry.provider.db.generic.tools.command;
-
-import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
-
-/**
- * The interface for all admin commands, eg, CreateRoleCmd.
- */
-public interface Command {
-  void execute(SentryGenericServiceClient client, String requestorName) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/CreateRoleCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/CreateRoleCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/CreateRoleCmd.java
deleted file mode 100644
index da60a64..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/CreateRoleCmd.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * 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.sentry.provider.db.generic.tools.command;
-
-import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
-
-/**
- * The class for admin command to create role.
- */
-public class CreateRoleCmd implements Command {
-
-  private String roleName;
-  private String component;
-
-  public CreateRoleCmd(String roleName, String component) {
-    this.roleName = roleName;
-    this.component = component;
-  }
-
-  @Override
-  public void execute(SentryGenericServiceClient client, String requestorName) throws Exception {
-    client.createRole(requestorName, roleName, component);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/DeleteRoleFromGroupCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/DeleteRoleFromGroupCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/DeleteRoleFromGroupCmd.java
deleted file mode 100644
index 95f39ea..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/DeleteRoleFromGroupCmd.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * 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.sentry.provider.db.generic.tools.command;
-
-import com.google.common.collect.Sets;
-import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
-import org.apache.sentry.provider.db.tools.SentryShellCommon;
-
-import java.util.Set;
-
-/**
- * Command for deleting groups from a role.
- */
-public class DeleteRoleFromGroupCmd implements Command {
-
-  private String roleName;
-  private String groups;
-  private String component;
-
-  public DeleteRoleFromGroupCmd(String roleName, String groups, String component) {
-    this.groups = groups;
-    this.roleName = roleName;
-    this.component = component;
-  }
-
-  @Override
-  public void execute(SentryGenericServiceClient client, String requestorName) throws Exception {
-    Set<String> groupSet = Sets.newHashSet(groups.split(SentryShellCommon.GROUP_SPLIT_CHAR));
-    client.deleteRoleToGroups(requestorName, roleName, component, groupSet);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/DropRoleCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/DropRoleCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/DropRoleCmd.java
deleted file mode 100644
index ac2a328..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/DropRoleCmd.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * 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.sentry.provider.db.generic.tools.command;
-
-import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
-
-/**
- * The class for admin command to drop role.
- */
-public class DropRoleCmd implements Command {
-
-  private String roleName;
-  private String component;
-
-  public DropRoleCmd(String roleName, String component) {
-    this.roleName = roleName;
-    this.component = component;
-  }
-
-  @Override
-  public void execute(SentryGenericServiceClient client, String requestorName) throws Exception {
-    client.dropRole(requestorName, roleName, component);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GenericShellCommand.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GenericShellCommand.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GenericShellCommand.java
new file mode 100644
index 0000000..5a3baad
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GenericShellCommand.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sentry.provider.db.generic.tools.command;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.sentry.core.common.exception.SentryUserException;
+import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
+import org.apache.sentry.provider.db.generic.service.thrift.TSentryPrivilege;
+import org.apache.sentry.provider.db.generic.service.thrift.TSentryRole;
+import org.apache.sentry.provider.db.tools.SentryShellCommon;
+import org.apache.sentry.provider.db.tools.ShellCommand;
+
+import com.google.common.collect.Sets;
+
+/**
+ * The ShellCommand implementation for the Generic clients
+ */
+public class GenericShellCommand implements ShellCommand {
+
+  private final SentryGenericServiceClient client;
+  private final String component;
+  private final TSentryPrivilegeConverter converter;
+  private final String serviceName;
+
+  public GenericShellCommand(SentryGenericServiceClient client, String component, String serviceName,
+                             TSentryPrivilegeConverter converter) {
+    this.client = client;
+    this.component = component;
+    this.serviceName = serviceName;
+    this.converter = converter;
+  }
+
+  public void createRole(String requestorName, String roleName) throws SentryUserException {
+    client.createRole(requestorName, roleName, component);
+  }
+
+  public void dropRole(String requestorName, String roleName) throws SentryUserException {
+    client.dropRole(requestorName, roleName, component);
+  }
+
+  public void grantPrivilegeToRole(String requestorName, String roleName, String privilege) throws SentryUserException {
+    TSentryPrivilege sentryPrivilege = converter.fromString(privilege);
+    client.grantPrivilege(requestorName, roleName, component, sentryPrivilege);
+  }
+
+  public void grantRoleToGroups(String requestorName, String roleName, String groups) throws SentryUserException {
+    Set<String> groupSet = Sets.newHashSet(groups.split(SentryShellCommon.GROUP_SPLIT_CHAR));
+    client.addRoleToGroups(requestorName, roleName, component, groupSet);
+  }
+
+  public void revokePrivilegeFromRole(String requestorName, String roleName, String privilege) throws SentryUserException {
+    TSentryPrivilege sentryPrivilege = converter.fromString(privilege);
+    client.revokePrivilege(requestorName, roleName, component, sentryPrivilege);
+  }
+
+  public void revokeRoleFromGroups(String requestorName, String roleName, String groups) throws SentryUserException {
+    Set<String> groupSet = Sets.newHashSet(groups.split(SentryShellCommon.GROUP_SPLIT_CHAR));
+    client.deleteRoleToGroups(requestorName, roleName, component, groupSet);
+  }
+
+  public List<String> listRoles(String requestorName, String roleName, String group) throws SentryUserException {
+    Set<TSentryRole> roles;
+    if (StringUtils.isEmpty(group)) {
+      roles = client.listAllRoles(requestorName, component);
+    } else {
+      roles = client.listRolesByGroupName(requestorName, group, component);
+    }
+
+    List<String> result = new ArrayList<>();
+    if (roles != null) {
+      for (TSentryRole role : roles) {
+        result.add(role.getRoleName());
+      }
+    }
+
+    return result;
+  }
+
+  public List<String> listPrivileges(String requestorName, String roleName) throws SentryUserException {
+    Set<TSentryPrivilege> privileges = client
+        .listPrivilegesByRoleName(requestorName, roleName, component, serviceName);
+
+    List<String> result = new ArrayList<>();
+    if (privileges != null) {
+      for (TSentryPrivilege privilege : privileges) {
+        String privilegeStr = converter.toString(privilege);
+        result.add(privilegeStr);
+      }
+    }
+
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GrantPrivilegeToRoleCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GrantPrivilegeToRoleCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GrantPrivilegeToRoleCmd.java
deleted file mode 100644
index 634bb42..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/GrantPrivilegeToRoleCmd.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.sentry.provider.db.generic.tools.command;
-
-import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
-import org.apache.sentry.provider.db.generic.service.thrift.TSentryPrivilege;
-
-/**
- * The class for admin command to grant privilege to role.
- */
-public class GrantPrivilegeToRoleCmd implements Command {
-
-  private String roleName;
-  private String component;
-  private String privilegeStr;
-  private TSentryPrivilegeConverter converter;
-
-  public GrantPrivilegeToRoleCmd(String roleName, String component, String privilegeStr,
-      TSentryPrivilegeConverter converter) {
-    this.roleName = roleName;
-    this.component = component;
-    this.privilegeStr = privilegeStr;
-    this.converter = converter;
-  }
-
-  @Override
-  public void execute(SentryGenericServiceClient client, String requestorName) throws Exception {
-    TSentryPrivilege privilege = converter.fromString(privilegeStr);
-    client.grantPrivilege(requestorName, roleName, component, privilege);
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/ListPrivilegesByRoleCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/ListPrivilegesByRoleCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/ListPrivilegesByRoleCmd.java
deleted file mode 100644
index ce6db3a..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/ListPrivilegesByRoleCmd.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * 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.sentry.provider.db.generic.tools.command;
-
-import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
-import org.apache.sentry.provider.db.generic.service.thrift.TSentryPrivilege;
-
-import java.util.Set;
-
-/**
- * The class for admin command to list privileges by role.
- */
-public class ListPrivilegesByRoleCmd implements Command {
-
-  private String roleName;
-  private String component;
-  private String serviceName;
-  private TSentryPrivilegeConverter converter;
-
-  public ListPrivilegesByRoleCmd(String roleName, String component, String serviceName,
-      TSentryPrivilegeConverter converter) {
-    this.roleName = roleName;
-    this.component = component;
-    this.serviceName = serviceName;
-    this.converter = converter;
-  }
-
-  @Override
-  public void execute(SentryGenericServiceClient client, String requestorName) throws Exception {
-    Set<TSentryPrivilege> privileges = client
-            .listPrivilegesByRoleName(requestorName, roleName, component, serviceName);
-    if (privileges != null) {
-      for (TSentryPrivilege privilege : privileges) {
-        String privilegeStr = converter.toString(privilege);
-        System.out.println(privilegeStr);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/ListRolesCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/ListRolesCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/ListRolesCmd.java
deleted file mode 100644
index 6b68d06..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/ListRolesCmd.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * 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.sentry.provider.db.generic.tools.command;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
-import org.apache.sentry.provider.db.generic.service.thrift.TSentryRole;
-
-import java.util.Set;
-
-/**
- * The class for admin command to list roles.
- */
-public class ListRolesCmd implements Command {
-
-  private String groupName;
-  private String component;
-
-  public ListRolesCmd(String groupName, String component) {
-    this.groupName = groupName;
-    this.component = component;
-  }
-
-  @Override
-  public void execute(SentryGenericServiceClient client, String requestorName) throws Exception {
-    Set<TSentryRole> roles;
-    if (StringUtils.isEmpty(groupName)) {
-      roles = client.listAllRoles(requestorName, component);
-    } else {
-      roles = client.listRolesByGroupName(requestorName, groupName, component);
-    }
-    if (roles != null) {
-      for (TSentryRole role : roles) {
-        System.out.println(role.getRoleName());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/RevokePrivilegeFromRoleCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/RevokePrivilegeFromRoleCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/RevokePrivilegeFromRoleCmd.java
deleted file mode 100644
index 3e42e60..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/RevokePrivilegeFromRoleCmd.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.sentry.provider.db.generic.tools.command;
-
-import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
-import org.apache.sentry.provider.db.generic.service.thrift.TSentryPrivilege;
-
-/**
- * The class for admin command to revoke privileges from role.
- */
-public class RevokePrivilegeFromRoleCmd implements Command {
-
-  private String roleName;
-  private String component;
-  private String privilegeStr;
-  private TSentryPrivilegeConverter converter;
-
-  public RevokePrivilegeFromRoleCmd(String roleName, String component, String privilegeStr,
-      TSentryPrivilegeConverter converter) {
-    this.roleName = roleName;
-    this.component = component;
-    this.privilegeStr = privilegeStr;
-    this.converter = converter;
-  }
-
-  @Override
-  public void execute(SentryGenericServiceClient client, String requestorName) throws Exception {
-    TSentryPrivilege privilege = converter.fromString(privilegeStr);
-    client.revokePrivilege(requestorName, roleName, component, privilege);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/TSentryPrivilegeConverter.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/TSentryPrivilegeConverter.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/TSentryPrivilegeConverter.java
index ab44895..0bfbc44 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/TSentryPrivilegeConverter.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/command/TSentryPrivilegeConverter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.sentry.provider.db.generic.tools.command;
 
+import org.apache.sentry.core.common.exception.SentryUserException;
 import org.apache.sentry.provider.db.generic.service.thrift.TSentryPrivilege;
 
 public interface TSentryPrivilegeConverter {
@@ -24,7 +25,7 @@ public interface TSentryPrivilegeConverter {
   /**
    * Convert string to privilege
    */
-  TSentryPrivilege fromString(String privilegeStr) throws Exception;
+  TSentryPrivilege fromString(String privilegeStr) throws SentryUserException;
 
   /**
    * Convert privilege to string

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/SentryShellHive.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/SentryShellHive.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/SentryShellHive.java
index 09f17ed..226d58d 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/SentryShellHive.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/SentryShellHive.java
@@ -18,6 +18,8 @@
 
 package org.apache.sentry.provider.db.tools;
 
+import java.util.List;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -39,39 +41,41 @@ public class SentryShellHive extends SentryShellCommon {
   private static final Logger LOGGER = LoggerFactory.getLogger(SentryShellHive.class);
 
   public void run() throws Exception {
-    Command command = null;
 
     try(SentryPolicyServiceClient client =
                 SentryServiceClientFactory.create(getSentryConf())) {
       UserGroupInformation ugi = UserGroupInformation.getLoginUser();
       String requestorName = ugi.getShortUserName();
+      ShellCommand command = new HiveShellCommand(client);
+
+      // check the requestor name
+      if (StringUtils.isEmpty(requestorName)) {
+        // The exception message will be recorded in the log file.
+        throw new Exception("The requestor name is empty.");
+      }
 
       if (isCreateRole) {
-        command = new CreateRoleCmd(roleName);
+        command.createRole(requestorName, roleName);
       } else if (isDropRole) {
-        command = new DropRoleCmd(roleName);
+        command.dropRole(requestorName, roleName);
       } else if (isAddRoleGroup) {
-        command = new GrantRoleToGroupsCmd(roleName, groupName);
+        command.grantRoleToGroups(requestorName, roleName, groupName);
       } else if (isDeleteRoleGroup) {
-        command = new RevokeRoleFromGroupsCmd(roleName, groupName);
+        command.revokeRoleFromGroups(requestorName, roleName, groupName);
       } else if (isGrantPrivilegeRole) {
-        command = new GrantPrivilegeToRoleCmd(roleName, privilegeStr);
+        command.grantPrivilegeToRole(requestorName, roleName, privilegeStr);
       } else if (isRevokePrivilegeRole) {
-        command = new RevokePrivilegeFromRoleCmd(roleName, privilegeStr);
+        command.revokePrivilegeFromRole(requestorName, roleName, privilegeStr);
       } else if (isListRole) {
-        command = new ListRolesCmd(groupName);
+        List<String> roles = command.listRoles(requestorName, roleName, groupName);
+        for (String role : roles) {
+          System.out.println(role);
+        }
       } else if (isListPrivilege) {
-        command = new ListPrivilegesCmd(roleName);
-      }
-
-      // check the requestor name
-      if (StringUtils.isEmpty(requestorName)) {
-        // The exception message will be recoreded in log file.
-        throw new Exception("The requestor name is empty.");
-      }
-
-      if (command != null) {
-        command.execute(client, requestorName);
+        List<String> privileges = command.listPrivileges(requestorName, roleName);
+        for (String privilege : privileges) {
+          System.out.println(privilege);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/ShellCommand.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/ShellCommand.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/ShellCommand.java
new file mode 100644
index 0000000..ec751ec
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/ShellCommand.java
@@ -0,0 +1,44 @@
+/**
+ * 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.sentry.provider.db.tools;
+
+import java.util.List;
+
+import org.apache.sentry.core.common.exception.SentryUserException;
+
+/**
+ * The interface for all admin commands, eg, CreateRoleCmd. It is independent of the underlying mechanism (i.e. Generic or Hive)
+ */
+public interface ShellCommand {
+
+    void createRole(String requestorName, String roleName) throws SentryUserException;
+
+    void dropRole(String requestorName, String roleName) throws SentryUserException;
+
+    void grantPrivilegeToRole(String requestorName, String roleName, String privilege) throws SentryUserException;
+
+    void grantRoleToGroups(String requestorName, String roleName, String groups) throws SentryUserException;
+
+    void revokePrivilegeFromRole(String requestorName, String roleName, String privilege) throws SentryUserException;
+
+    void revokeRoleFromGroups(String requestorName, String roleName, String groups) throws SentryUserException;
+
+    List<String> listRoles(String requestorName, String roleName, String group) throws SentryUserException;
+
+    List<String> listPrivileges(String requestorName, String roleName) throws SentryUserException;
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/Command.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/Command.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/Command.java
deleted file mode 100644
index 79aed49..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/Command.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * 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.sentry.provider.db.tools.command.hive;
-
-import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-
-/**
- * The interface for all admin commands, eg, CreateRoleCmd.
- */
-public interface Command {
-  void execute(SentryPolicyServiceClient client, String requestorName) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CommandUtil.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CommandUtil.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CommandUtil.java
index b6f4140..3f0b5fa 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CommandUtil.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CommandUtil.java
@@ -31,7 +31,7 @@ public final class CommandUtil {
 
   // check the privilege value for the specific privilege scope
   // eg, for the table scope, server and database can't be empty
-  public static void validatePrivilegeHierarchy(TSentryPrivilege tSentryPrivilege) throws Exception {
+  public static void validatePrivilegeHierarchy(TSentryPrivilege tSentryPrivilege) throws IllegalArgumentException {
     String serverName = tSentryPrivilege.getServerName();
     String dbName = tSentryPrivilege.getDbName();
     String tableName = tSentryPrivilege.getTableName();

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CreateRoleCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CreateRoleCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CreateRoleCmd.java
deleted file mode 100644
index 5a4834a..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CreateRoleCmd.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * 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.sentry.provider.db.tools.command.hive;
-
-import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-
-/**
- * The class for admin command to create role.
- */
-public class CreateRoleCmd implements Command {
-
-  private String roleName;
-
-  public CreateRoleCmd(String roleName) {
-    this.roleName = roleName;
-  }
-
-  @Override
-  public void execute(SentryPolicyServiceClient client, String requestorName) throws Exception {
-    client.createRole(requestorName, roleName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/DropRoleCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/DropRoleCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/DropRoleCmd.java
deleted file mode 100644
index facec0e..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/DropRoleCmd.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * 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.sentry.provider.db.tools.command.hive;
-
-import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-
-/**
- * The class for admin command to drop role.
- */
-public class DropRoleCmd implements Command {
-
-  private String roleName;
-
-  public DropRoleCmd(String roleName) {
-    this.roleName = roleName;
-  }
-
-  @Override
-  public void execute(SentryPolicyServiceClient client, String requestorName) throws Exception {
-    client.dropRole(requestorName, roleName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantPrivilegeToRoleCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantPrivilegeToRoleCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantPrivilegeToRoleCmd.java
deleted file mode 100644
index f530c00..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantPrivilegeToRoleCmd.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.sentry.provider.db.tools.command.hive;
-
-import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
-import org.apache.sentry.service.thrift.SentryServiceUtil;
-
-/**
- * The class for admin command to grant privilege to role.
- */
-public class GrantPrivilegeToRoleCmd implements Command {
-
-  private String roleName;
-  private String privilegeStr;
-
-  public GrantPrivilegeToRoleCmd(String roleName, String privilegeStr) {
-    this.roleName = roleName;
-    this.privilegeStr = privilegeStr;
-  }
-
-  @Override
-  public void execute(SentryPolicyServiceClient client, String requestorName) throws Exception {
-    TSentryPrivilege tSentryPrivilege = SentryServiceUtil.convertToTSentryPrivilege(privilegeStr);
-    CommandUtil.validatePrivilegeHierarchy(tSentryPrivilege);
-    client.grantPrivilege(requestorName, roleName, tSentryPrivilege);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantRoleToGroupsCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantRoleToGroupsCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantRoleToGroupsCmd.java
deleted file mode 100644
index 07a3de4..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantRoleToGroupsCmd.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * 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.sentry.provider.db.tools.command.hive;
-
-import com.google.common.collect.Sets;
-import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-import org.apache.sentry.provider.db.tools.SentryShellCommon;
-
-import java.util.Set;
-
-/**
- * The class for admin command to grant role to group.
- */
-public class GrantRoleToGroupsCmd implements Command {
-
-  private String roleName;
-  private String groupNamesStr;
-
-  public GrantRoleToGroupsCmd(String roleName, String groupNamesStr) {
-    this.roleName = roleName;
-    this.groupNamesStr = groupNamesStr;
-  }
-
-  @Override
-  public void execute(SentryPolicyServiceClient client, String requestorName) throws Exception {
-    Set<String> groups = Sets.newHashSet(groupNamesStr.split(SentryShellCommon.GROUP_SPLIT_CHAR));
-    client.grantRoleToGroups(requestorName, roleName, groups);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/HiveShellCommand.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/HiveShellCommand.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/HiveShellCommand.java
new file mode 100644
index 0000000..8451d8b
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/HiveShellCommand.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sentry.provider.db.tools.command.hive;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.sentry.core.common.exception.SentryUserException;
+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.provider.db.tools.SentryShellCommon;
+import org.apache.sentry.provider.db.tools.ShellCommand;
+import org.apache.sentry.service.thrift.SentryServiceUtil;
+
+import com.google.common.collect.Sets;
+
+/**
+ * The ShellCommand implementation for Hive.
+ */
+public class HiveShellCommand implements ShellCommand {
+
+  private final SentryPolicyServiceClient client;
+
+  public HiveShellCommand(SentryPolicyServiceClient client) {
+    this.client = client;
+  }
+
+  public void createRole(String requestorName, String roleName) throws SentryUserException {
+    client.createRole(requestorName, roleName);
+  }
+
+  public void dropRole(String requestorName, String roleName) throws SentryUserException {
+    client.dropRole(requestorName, roleName);
+  }
+
+  public void grantPrivilegeToRole(String requestorName, String roleName, String privilege) throws SentryUserException {
+    TSentryPrivilege tSentryPrivilege = SentryServiceUtil.convertToTSentryPrivilege(privilege);
+    CommandUtil.validatePrivilegeHierarchy(tSentryPrivilege);
+    client.grantPrivilege(requestorName, roleName, tSentryPrivilege);
+  }
+
+  public void grantRoleToGroups(String requestorName, String roleName, String groups) throws SentryUserException {
+    Set<String> groupSet = Sets.newHashSet(groups.split(SentryShellCommon.GROUP_SPLIT_CHAR));
+    client.grantRoleToGroups(requestorName, roleName, groupSet);
+  }
+
+  public void revokePrivilegeFromRole(String requestorName, String roleName, String privilege) throws SentryUserException {
+    TSentryPrivilege tSentryPrivilege = SentryServiceUtil.convertToTSentryPrivilege(privilege);
+    CommandUtil.validatePrivilegeHierarchy(tSentryPrivilege);
+    client.revokePrivilege(requestorName, roleName, tSentryPrivilege);
+  }
+
+  public void revokeRoleFromGroups(String requestorName, String roleName, String groups) throws SentryUserException {
+    Set<String> groupSet = Sets.newHashSet(groups.split(SentryShellCommon.GROUP_SPLIT_CHAR));
+    client.revokeRoleFromGroups(requestorName, roleName, groupSet);
+  }
+
+  public List<String> listRoles(String requestorName, String roleName, String group) throws SentryUserException {
+    Set<TSentryRole> roles;
+    if (StringUtils.isEmpty(group)) {
+      roles = client.listRoles(requestorName);
+    } else {
+      roles = client.listRolesByGroupName(requestorName, group);
+    }
+
+    List<String> result = new ArrayList<>();
+    if (roles != null) {
+      for (TSentryRole role : roles) {
+        result.add(role.getRoleName());
+      }
+    }
+
+    return result;
+  }
+
+  public List<String> listPrivileges(String requestorName, String roleName) throws SentryUserException {
+    Set<TSentryPrivilege> privileges = client
+        .listAllPrivilegesByRoleName(requestorName, roleName);
+
+    List<String> result = new ArrayList<>();
+    if (privileges != null) {
+      for (TSentryPrivilege privilege : privileges) {
+        String privilegeStr = SentryServiceUtil.convertTSentryPrivilegeToStr(privilege);
+        result.add(privilegeStr);
+      }
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/ListPrivilegesCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/ListPrivilegesCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/ListPrivilegesCmd.java
deleted file mode 100644
index 2cc4f71..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/ListPrivilegesCmd.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * 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.sentry.provider.db.tools.command.hive;
-
-import java.util.Set;
-
-import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
-import org.apache.sentry.service.thrift.SentryServiceUtil;
-
-/**
- * The class for admin command to list privileges.
- */
-public class ListPrivilegesCmd implements Command {
-
-  private String roleName;
-
-  public ListPrivilegesCmd(String roleName) {
-    this.roleName = roleName;
-  }
-
-  @Override
-  public void execute(SentryPolicyServiceClient client, String requestorName) throws Exception {
-    Set<TSentryPrivilege> privileges = client
-            .listAllPrivilegesByRoleName(requestorName, roleName);
-    if (privileges != null) {
-      for (TSentryPrivilege privilege : privileges) {
-        String privilegeStr = SentryServiceUtil.convertTSentryPrivilegeToStr(privilege);
-        System.out.println(privilegeStr);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/ListRolesCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/ListRolesCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/ListRolesCmd.java
deleted file mode 100644
index 283f2c0..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/ListRolesCmd.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * 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.sentry.provider.db.tools.command.hive;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-import org.apache.sentry.provider.db.service.thrift.TSentryRole;
-
-import java.util.Set;
-
-/**
- * The class for admin command to list roles.
- */
-public class ListRolesCmd implements Command {
-
-  private String groupName;
-
-  public ListRolesCmd(String groupName) {
-    this.groupName = groupName;
-  }
-
-  @Override
-  public void execute(SentryPolicyServiceClient client, String requestorName) throws Exception {
-    Set<TSentryRole> roles;
-    if (StringUtils.isEmpty(groupName)) {
-      roles = client.listRoles(requestorName);
-    } else {
-      roles = client.listRolesByGroupName(requestorName, groupName);
-    }
-    if (roles != null) {
-      for (TSentryRole role : roles) {
-        System.out.println(role.getRoleName());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/RevokePrivilegeFromRoleCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/RevokePrivilegeFromRoleCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/RevokePrivilegeFromRoleCmd.java
deleted file mode 100644
index 4acecee..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/RevokePrivilegeFromRoleCmd.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * 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.sentry.provider.db.tools.command.hive;
-
-import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
-import org.apache.sentry.service.thrift.SentryServiceUtil;
-
-/**
- * The class for admin command to revoke privileges from role.
- */
-public class RevokePrivilegeFromRoleCmd implements Command {
-
-  private String roleName;
-  private String privilegeStr;
-
-  public RevokePrivilegeFromRoleCmd(String roleName, String privilegeStr) {
-    this.roleName = roleName;
-    this.privilegeStr = privilegeStr;
-  }
-
-  @Override
-  public void execute(SentryPolicyServiceClient client, String requestorName) throws Exception {
-    TSentryPrivilege tSentryPrivilege = SentryServiceUtil.convertToTSentryPrivilege(privilegeStr);
-    CommandUtil.validatePrivilegeHierarchy(tSentryPrivilege);
-    client.revokePrivilege(requestorName, roleName, tSentryPrivilege);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/6d44cfd9/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/RevokeRoleFromGroupsCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/RevokeRoleFromGroupsCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/RevokeRoleFromGroupsCmd.java
deleted file mode 100644
index 86773ca..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/RevokeRoleFromGroupsCmd.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.sentry.provider.db.tools.command.hive;
-
-import com.google.common.collect.Sets;
-import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-
-import java.util.Set;
-
-/**
- * The class for admin command to revoke role from group.
- */
-public class RevokeRoleFromGroupsCmd implements Command {
-
-  private String roleName;
-  private String groupNamesStr;
-
-  public RevokeRoleFromGroupsCmd(String roleName, String groupNamesStr) {
-    this.roleName = roleName;
-    this.groupNamesStr = groupNamesStr;
-  }
-
-  @Override
-  public void execute(SentryPolicyServiceClient client, String requestorName) throws Exception {
-    Set<String> groups = Sets.newHashSet(groupNamesStr.split(CommandUtil.SPLIT_CHAR));
-    client.revokeRoleFromGroups(requestorName, roleName, groups);
-  }
-}