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/19 08:59:46 UTC

[05/26] sentry git commit: SENTRY-1210 - Refactor the SentryShellSolr and SentryShellKafka

SENTRY-1210 - Refactor the SentryShellSolr and SentryShellKafka


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

Branch: refs/heads/akolb-cli
Commit: d7a6667cb7d5c40bab753c254d01a5d4ba68de2e
Parents: 1749f7e
Author: Colm O hEigeartaigh <co...@apache.org>
Authored: Wed Oct 4 10:13:50 2017 +0100
Committer: Colm O hEigeartaigh <co...@apache.org>
Committed: Wed Oct 4 10:13:50 2017 +0100

----------------------------------------------------------------------
 bin/sentryShell                                 |   4 +-
 .../db/generic/tools/SentryShellGeneric.java    | 155 ++++++++++++++++++
 .../db/generic/tools/SentryShellKafka.java      | 115 -------------
 .../db/generic/tools/SentryShellSolr.java       | 114 -------------
 .../provider/db/tools/SentryShellCommon.java    |  51 +++---
 .../db/generic/tools/TestSentryShellKafka.java  | 164 +++++++++----------
 .../db/generic/tools/TestSentryShellSolr.java   | 162 +++++++++---------
 7 files changed, 348 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/d7a6667c/bin/sentryShell
----------------------------------------------------------------------
diff --git a/bin/sentryShell b/bin/sentryShell
index f066156..6fd2848 100755
--- a/bin/sentryShell
+++ b/bin/sentryShell
@@ -60,8 +60,8 @@ while [ $# -gt 0 ]; do    # Until you run out of parameters . . .
   if [[ "$1" = "-t" || "$1" = "--type" ]]; then
     case $2 in
       "hive") shell=org.apache.sentry.provider.db.tools.SentryShellHive ;;
-      "kafka") shell=org.apache.sentry.provider.db.generic.tools.SentryShellKafka ;;
-      "solr") shell=org.apache.sentry.provider.db.generic.tools.SentryShellSolr ;;
+      "kafka") shell=org.apache.sentry.provider.db.generic.tools.SentryShellGeneric ;;
+      "solr") shell=org.apache.sentry.provider.db.generic.tools.SentryShellGeneric ;;
       *) echo "Doesn't support the type $2!"; exit 1 ;;
     esac
   fi

http://git-wip-us.apache.org/repos/asf/sentry/blob/d7a6667c/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
new file mode 100644
index 0000000..49523a4
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellGeneric.java
@@ -0,0 +1,155 @@
+/**
+ * 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;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+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.TSentryPrivilegeConverter;
+import org.apache.sentry.provider.db.tools.SentryShellCommon;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * SentryShellGeneric is an admin tool, and responsible for the management of repository.
+ * The following commands are supported:
+ * create role, drop role, add group to role, grant privilege to role,
+ * revoke privilege from role, list roles, list privilege for role.
+ */
+public class SentryShellGeneric extends SentryShellCommon {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SentryShellGeneric.class);
+  private static final String KAFKA_SERVICE_NAME = "sentry.service.client.kafka.service.name";
+  private static final String SOLR_SERVICE_NAME = "sentry.service.client.solr.service.name";
+
+  @Override
+  public void run() throws Exception {
+    Command command = null;
+    String component = getComponent();
+    Configuration conf = getSentryConf();
+
+    String service = getService(conf);
+    try (SentryGenericServiceClient client =
+                SentryGenericServiceClientFactory.create(conf)) {
+      UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+      String requestorName = ugi.getShortUserName();
+
+      if (isCreateRole) {
+        command = new CreateRoleCmd(roleName, component);
+      } else if (isDropRole) {
+        command = new DropRoleCmd(roleName, component);
+      } else if (isAddRoleGroup) {
+        command = new AddRoleToGroupCmd(roleName, groupName, component);
+      } else if (isDeleteRoleGroup) {
+        command = new DeleteRoleFromGroupCmd(roleName, groupName, component);
+      } else if (isGrantPrivilegeRole) {
+        command = new GrantPrivilegeToRoleCmd(roleName, component,
+                privilegeStr, getPrivilegeConverter(component, service));
+      } else if (isRevokePrivilegeRole) {
+        command = new RevokePrivilegeFromRoleCmd(roleName, component,
+                privilegeStr, getPrivilegeConverter(component, service));
+      } else if (isListRole) {
+        command = new ListRolesCmd(groupName, component);
+      } else if (isListPrivilege) {
+        command = new ListPrivilegesByRoleCmd(roleName, component,
+                service, getPrivilegeConverter(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);
+      }
+    }
+  }
+
+  private String getComponent() throws Exception {
+    if (type == TYPE.kafka) {
+      return AuthorizationComponent.KAFKA;
+    } else if (type == TYPE.solr) {
+      return "SOLR";
+    }
+
+    throw new Exception("Invalid type specified for SentryShellGeneric: " + type);
+  }
+
+  private String getService(Configuration conf) throws Exception {
+    if (type == TYPE.kafka) {
+      return conf.get(KAFKA_SERVICE_NAME, "kafka1");
+    } else if (type == TYPE.solr) {
+      return conf.get(SOLR_SERVICE_NAME, "service1");
+    }
+
+    throw new Exception("Invalid type specified for SentryShellGeneric: " + type);
+  }
+
+  private TSentryPrivilegeConverter getPrivilegeConverter(String component, String service) throws Exception {
+    if (type == TYPE.kafka) {
+      return new KafkaTSentryPrivilegeConverter(component, service);
+    } else if (type == TYPE.solr) {
+      return new SolrTSentryPrivilegeConverter(component, service);
+    }
+
+    throw new Exception("Invalid type specified for SentryShellGeneric: " + type);
+  }
+
+  private Configuration getSentryConf() {
+    Configuration conf = new Configuration();
+    conf.addResource(new Path(confPath));
+    return conf;
+  }
+
+  public static void main(String[] args) throws Exception {
+    SentryShellGeneric sentryShell = new SentryShellGeneric();
+    try {
+      sentryShell.executeShell(args);
+    } catch (Exception e) {
+      LOGGER.error(e.getMessage(), e);
+      Throwable current = e;
+      // find the first printable message;
+      while (current != null && current.getMessage() == null) {
+        current = current.getCause();
+      }
+      String error = "";
+      if (current != null && current.getMessage() != null) {
+        error = "Message: " + current.getMessage();
+      }
+      System.out.println("The operation failed. " + error);
+      System.exit(1);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/d7a6667c/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellKafka.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellKafka.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellKafka.java
deleted file mode 100644
index f6e5d1b..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellKafka.java
+++ /dev/null
@@ -1,115 +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;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-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.*;
-import org.apache.sentry.provider.db.tools.SentryShellCommon;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * SentryShellKafka is an admin tool, and responsible for the management of repository.
- * The following commands are supported:
- * create role, drop role, add group to role, grant privilege to role,
- * revoke privilege from role, list roles, list privilege for role.
- */
-public class SentryShellKafka extends SentryShellCommon {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(SentryShellKafka.class);
-  public static final String KAFKA_SERVICE_NAME = "sentry.service.client.kafka.service.name";
-
-  @Override
-  public void run() throws Exception {
-    Command command = null;
-    String component = AuthorizationComponent.KAFKA;
-    Configuration conf = getSentryConf();
-
-    String service = conf.get(KAFKA_SERVICE_NAME, "kafka1");
-    try(SentryGenericServiceClient client =
-                SentryGenericServiceClientFactory.create(conf)) {
-      UserGroupInformation ugi = UserGroupInformation.getLoginUser();
-      String requestorName = ugi.getShortUserName();
-
-      if (isCreateRole) {
-        command = new CreateRoleCmd(roleName, component);
-      } else if (isDropRole) {
-        command = new DropRoleCmd(roleName, component);
-      } else if (isAddRoleGroup) {
-        command = new AddRoleToGroupCmd(roleName, groupName, component);
-      } else if (isDeleteRoleGroup) {
-        command = new DeleteRoleFromGroupCmd(roleName, groupName, component);
-      } else if (isGrantPrivilegeRole) {
-        command = new GrantPrivilegeToRoleCmd(roleName, component,
-                privilegeStr, new KafkaTSentryPrivilegeConverter(component, service));
-      } else if (isRevokePrivilegeRole) {
-        command = new RevokePrivilegeFromRoleCmd(roleName, component,
-                privilegeStr, new KafkaTSentryPrivilegeConverter(component, service));
-      } else if (isListRole) {
-        command = new ListRolesCmd(groupName, component);
-      } else if (isListPrivilege) {
-        command = new ListPrivilegesByRoleCmd(roleName, component,
-                service, new KafkaTSentryPrivilegeConverter(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);
-      }
-    }
-  }
-
-  private Configuration getSentryConf() {
-    Configuration conf = new Configuration();
-    conf.addResource(new Path(confPath));
-    return conf;
-  }
-
-  public static void main(String[] args) throws Exception {
-    SentryShellKafka sentryShell = new SentryShellKafka();
-    try {
-      sentryShell.executeShell(args);
-    } catch (Exception e) {
-      LOGGER.error(e.getMessage(), e);
-      Throwable current = e;
-      // find the first printable message;
-      while (current != null && current.getMessage() == null) {
-        current = current.getCause();
-      }
-      String error = "";
-      if (current != null && current.getMessage() != null) {
-        error = "Message: " + current.getMessage();
-      }
-      System.out.println("The operation failed. " + error);
-      System.exit(1);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/d7a6667c/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellSolr.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellSolr.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellSolr.java
deleted file mode 100644
index 5385f7d..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryShellSolr.java
+++ /dev/null
@@ -1,114 +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;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-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.*;
-import org.apache.sentry.provider.db.tools.SentryShellCommon;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * SentryShellSolr is an admin tool, and responsible for the management of repository.
- * The following commands are supported:
- * create role, drop role, add group to role, grant privilege to role,
- * revoke privilege from role, list roles, list privilege for role.
- */
-public class SentryShellSolr extends SentryShellCommon {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(SentryShellSolr.class);
-  public static final String SOLR_SERVICE_NAME = "sentry.service.client.solr.service.name";
-
-  @Override
-  public void run() throws Exception {
-    Command command = null;
-    String component = "SOLR";
-    Configuration conf = getSentryConf();
-
-    String service = conf.get(SOLR_SERVICE_NAME, "service1");
-    try(SentryGenericServiceClient client =
-                SentryGenericServiceClientFactory.create(conf)) {
-      UserGroupInformation ugi = UserGroupInformation.getLoginUser();
-      String requestorName = ugi.getShortUserName();
-
-      if (isCreateRole) {
-        command = new CreateRoleCmd(roleName, component);
-      } else if (isDropRole) {
-        command = new DropRoleCmd(roleName, component);
-      } else if (isAddRoleGroup) {
-        command = new AddRoleToGroupCmd(roleName, groupName, component);
-      } else if (isDeleteRoleGroup) {
-        command = new DeleteRoleFromGroupCmd(roleName, groupName, component);
-      } else if (isGrantPrivilegeRole) {
-        command = new GrantPrivilegeToRoleCmd(roleName, component,
-                privilegeStr, new SolrTSentryPrivilegeConverter(component, service));
-      } else if (isRevokePrivilegeRole) {
-        command = new RevokePrivilegeFromRoleCmd(roleName, component,
-                privilegeStr, new SolrTSentryPrivilegeConverter(component, service));
-      } else if (isListRole) {
-        command = new ListRolesCmd(groupName, component);
-      } else if (isListPrivilege) {
-        command = new ListPrivilegesByRoleCmd(roleName, component,
-                service, new SolrTSentryPrivilegeConverter(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);
-      }
-    }
-  }
-
-  private Configuration getSentryConf() {
-    Configuration conf = new Configuration();
-    conf.addResource(new Path(confPath));
-    return conf;
-  }
-
-  public static void main(String[] args) throws Exception {
-    SentryShellSolr sentryShell = new SentryShellSolr();
-    try {
-      sentryShell.executeShell(args);
-    } catch (Exception e) {
-      LOGGER.error(e.getMessage(), e);
-      Throwable current = e;
-      // find the first printable message;
-      while (current != null && current.getMessage() == null) {
-        current = current.getCause();
-      }
-      String error = "";
-      if (current != null && current.getMessage() != null) {
-        error = "Message: " + current.getMessage();
-      }
-      System.out.println("The operation failed. " + error);
-      System.exit(1);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/d7a6667c/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/SentryShellCommon.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/SentryShellCommon.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/SentryShellCommon.java
index 6ddc1de..b2b6f4f 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/SentryShellCommon.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/SentryShellCommon.java
@@ -36,33 +36,36 @@ import org.apache.commons.lang.StringUtils;
  */
 abstract public class SentryShellCommon {
 
+  public enum TYPE { kafka, hive, solr };
+
+  public static final String OPTION_DESC_HELP = "Shell usage";
+  public static final String OPTION_DESC_CONF = "sentry-site file path";
+  public static final String OPTION_DESC_ROLE_NAME = "Role name";
+  public static final String OPTION_DESC_GROUP_NAME = "Group name";
+  public static final String OPTION_DESC_PRIVILEGE = "Privilege string";
+  public static final String PREFIX_MESSAGE_MISSING_OPTION = "Missing required option: ";
+
+  public static final String GROUP_SPLIT_CHAR = ",";
+
   protected String roleName;
   protected String groupName;
   protected String privilegeStr;
   protected String confPath;
   // flag for the command
-  protected boolean isCreateRole = false;
-  protected boolean isDropRole = false;
-  protected boolean isAddRoleGroup = false;
-  protected boolean isDeleteRoleGroup = false;
-  protected boolean isGrantPrivilegeRole = false;
-  protected boolean isRevokePrivilegeRole = false;
-  protected boolean isListRole = false;
-  protected boolean isListPrivilege = false;
-  protected boolean isPrintHelp = false;
+  protected boolean isCreateRole;
+  protected boolean isDropRole;
+  protected boolean isAddRoleGroup;
+  protected boolean isDeleteRoleGroup;
+  protected boolean isGrantPrivilegeRole;
+  protected boolean isRevokePrivilegeRole;
+  protected boolean isListRole;
+  protected boolean isListPrivilege;
+  protected boolean isPrintHelp;
   // flag for the parameter check
-  protected boolean roleNameRequired = false;
-  protected boolean groupNameRequired = false;
-  protected boolean privilegeStrRequired = false;
-
-  public final static String OPTION_DESC_HELP = "Shell usage";
-  public final static String OPTION_DESC_CONF = "sentry-site file path";
-  public final static String OPTION_DESC_ROLE_NAME = "Role name";
-  public final static String OPTION_DESC_GROUP_NAME = "Group name";
-  public final static String OPTION_DESC_PRIVILEGE = "Privilege string";
-  public final static String PREFIX_MESSAGE_MISSING_OPTION = "Missing required option: ";
-
-  public final static String GROUP_SPLIT_CHAR = ",";
+  protected boolean roleNameRequired;
+  protected boolean groupNameRequired;
+  protected boolean privilegeStrRequired;
+  protected TYPE type;
 
   /**
    * parse arguments
@@ -77,7 +80,7 @@ abstract public class SentryShellCommon {
    *   -rpr,--revoke_privilege_role -r <rolename>  -p <privilege> revoke privilege from role
    *   -lr,--list_role              -g <groupname>                list roles for group
    *   -lp,--list_privilege         -r <rolename>                 list privilege for role
-   *   -t,--type                    <typeame>                     the shell for hive model or generic model
+   *   -t,--type                    <typename>                    the shell for hive model or generic model
    * </pre>
    *
    * @param args
@@ -135,7 +138,7 @@ abstract public class SentryShellCommon {
     rOpt.setRequired(false);
     simpleShellOptions.addOption(rOpt);
 
-    // this argument should be parsed in the bin/sentryShell
+    // this argument should also be parsed in the bin/sentryShell
     Option tOpt = new Option("t", "type", true, "[hive|solr|sqoop|.....]");
     tOpt.setRequired(false);
     simpleShellOptions.addOption(tOpt);
@@ -206,6 +209,8 @@ abstract public class SentryShellCommon {
           roleNameRequired = true;
         } else if (opt.getOpt().equals("conf")) {
           confPath = opt.getValue();
+        } else if (opt.getOpt().equals("t")) {
+          type = TYPE.valueOf(opt.getValue());
         }
       }
       checkRequiredParameter(roleNameRequired, roleName, OPTION_DESC_ROLE_NAME);

http://git-wip-us.apache.org/repos/asf/sentry/blob/d7a6667c/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 7db5426..183e83d 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
@@ -80,32 +80,32 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
       @Override
       public void runTestAsSubject() throws Exception {
         // test: create role with -cr
-        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        SentryShellKafka.main(args);
+        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric.main(args);
         // test: create role with --create_role
         args = new String[] { "--create_role", "-r", TEST_ROLE_NAME_2, "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellKafka.main(args);
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric.main(args);
 
         // validate the result, list roles with -lr
-        args = new String[] { "-lr", "-conf", confPath.getAbsolutePath() };
-        SentryShellKafka sentryShell = new SentryShellKafka();
+        args = new String[] { "-lr", "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         Set<String> roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1, TEST_ROLE_NAME_2);
 
         // validate the result, list roles with --list_role
-        args = new String[] { "--list_role", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "--list_role", "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1, TEST_ROLE_NAME_2);
 
         // test: drop role with -dr
-        args = new String[] { "-dr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        SentryShellKafka.main(args);
+        args = new String[] { "-dr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric.main(args);
         // test: drop role with --drop_role
         args = new String[] { "--drop_role", "-r", TEST_ROLE_NAME_2, "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellKafka.main(args);
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric.main(args);
 
         // validate the result
         Set<TSentryRole> roles = client.listAllRoles(requestorName, KAFKA);
@@ -129,51 +129,51 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
         client.createRole(requestorName, TEST_ROLE_NAME_2, KAFKA);
         // test: add role to group with -arg
         String[] args = { "-arg", "-r", TEST_ROLE_NAME_1, "-g", TEST_GROUP_1, "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellKafka.main(args);
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric.main(args);
         // test: add role to multiple groups
         args = new String[] { "-arg", "-r", TEST_ROLE_NAME_1, "-g", TEST_GROUP_2 + "," + TEST_GROUP_3,
             "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellKafka.main(args);
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric.main(args);
         // test: add role to group with --add_role_group
         args = new String[] { "--add_role_group", "-r", TEST_ROLE_NAME_2, "-g", TEST_GROUP_1,
             "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellKafka.main(args);
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric.main(args);
 
         // validate the result list roles with -lr and -g
-        args = new String[] { "-lr", "-g", TEST_GROUP_1, "-conf", confPath.getAbsolutePath() };
-        SentryShellKafka sentryShell = new SentryShellKafka();
+        args = new String[] { "-lr", "-g", TEST_GROUP_1, "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         Set<String> roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1, TEST_ROLE_NAME_2);
 
         // list roles with --list_role and -g
         args = new String[] { "--list_role", "-g", TEST_GROUP_2, "-conf",
-            confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1);
 
         args = new String[] { "--list_role", "-g", TEST_GROUP_3, "-conf",
-            confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1);
 
         // test: delete role from group with -drg
         args = new String[] { "-drg", "-r", TEST_ROLE_NAME_1, "-g", TEST_GROUP_1, "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellKafka.main(args);
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric.main(args);
         // test: delete role to multiple groups
         args = new String[] { "-drg", "-r", TEST_ROLE_NAME_1, "-g", TEST_GROUP_2 + "," + TEST_GROUP_3,
             "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellKafka.main(args);
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric.main(args);
         // test: delete role from group with --delete_role_group
         args = new String[] { "--delete_role_group", "-r", TEST_ROLE_NAME_2, "-g", TEST_GROUP_1,
-            "-conf", confPath.getAbsolutePath() };
-        SentryShellKafka.main(args);
+            "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric.main(args);
 
         // validate the result
         Set<TSentryRole> roles = client.listRolesByGroupName(requestorName, TEST_GROUP_1, KAFKA);
@@ -199,17 +199,17 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
         client.createRole(requestorName, TEST_ROLE_NAME_1, KAFKA);
         // add role to a group (lower case)
         String[] args = {"-arg", "-r", TEST_ROLE_NAME_1, "-g", "group1", "-conf",
-            confPath.getAbsolutePath()};
-        SentryShellKafka.main(args);
+            confPath.getAbsolutePath(), "-t", "kafka"};
+        SentryShellGeneric.main(args);
 
         // validate the roles when group name is same case as above
-        args = new String[]{"-lr", "-g", "group1", "-conf", confPath.getAbsolutePath()};
-        SentryShellKafka sentryShell = new SentryShellKafka();
+        args = new String[]{"-lr", "-g", "group1", "-conf", confPath.getAbsolutePath(), "-t", "kafka"};
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         Set<String> roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1);
 
         // roles should be empty when group name is different case than above
-        args = new String[]{"-lr", "-g", "GROUP1", "-conf", confPath.getAbsolutePath()};
+        args = new String[]{"-lr", "-g", "GROUP1", "-conf", confPath.getAbsolutePath(), "-t", "kafka"};
         roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames);
       }
@@ -247,13 +247,13 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
           // test: grant privilege to role
           String [] args = new String [] { grant(shortOption), "-r", TEST_ROLE_NAME_1, "-p",
             privs[ i ],
-            "-conf", confPath.getAbsolutePath() };
-          SentryShellKafka.main(args);
+            "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+          SentryShellGeneric.main(args);
         }
 
         // test the list privilege
-        String [] args = new String[] { list(shortOption), "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        SentryShellKafka sentryShell = new SentryShellKafka();
+        String [] args = new String[] { list(shortOption), "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         Set<String> privilegeStrs = getShellResultWithOSRedirect(sentryShell, args, true);
 
         assertEquals("Incorrect number of privileges", privs.length, privilegeStrs.size());
@@ -264,8 +264,8 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
         for (int i = 0; i < privs.length; ++i) {
           args = new String[] { revoke(shortOption), "-r", TEST_ROLE_NAME_1, "-p",
             privs[ i ], "-conf",
-            confPath.getAbsolutePath() };
-          SentryShellKafka.main(args);
+            confPath.getAbsolutePath(), "-t", "kafka" };
+          SentryShellGeneric.main(args);
           Set<TSentryPrivilege> privileges = client.listPrivilegesByRoleName(requestorName,
             TEST_ROLE_NAME_1, KAFKA, service);
           assertEquals("Incorrect number of privileges. Received privileges: " + Arrays.toString(privileges.toArray()), privs.length - (i + 1), privileges.size());
@@ -297,8 +297,8 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
       public void runTestAsSubject() throws Exception {
         client.createRole(requestorName, TEST_ROLE_NAME_1, KAFKA);
         // test: create duplicate role with -cr
-        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        SentryShellKafka sentryShell = new SentryShellKafka();
+        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for creating duplicate role");
@@ -309,8 +309,8 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
         }
 
         // test: drop non-exist role with -dr
-        args = new String[] { "-dr", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-dr", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for dropping non-exist role");
@@ -322,8 +322,8 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
 
         // test: add non-exist role to group with -arg
         args = new String[] { "-arg", "-r", TEST_ROLE_NAME_2, "-g", "testGroup1", "-conf",
-            confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for granting non-exist role to group");
@@ -335,8 +335,8 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
 
         // test: drop group from non-exist role with -drg
         args = new String[] { "-drg", "-r", TEST_ROLE_NAME_2, "-g", "testGroup1", "-conf",
-            confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for drop group from non-exist role");
@@ -348,8 +348,8 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
 
         // test: grant privilege to role with the error privilege format
         args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-p", "serverserver1->action=all",
-            "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+            "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for the error privilege format, invalid key value.");
@@ -362,8 +362,8 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
         // test: grant privilege to role with the error privilege hierarchy
         args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-p",
             "consumergroup=cg1->host=h1->action=create", "-conf",
-            confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+            confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for the error privilege format, invalid key value.");
@@ -387,62 +387,62 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
         String strOptionConf = "conf";
         client.createRole(requestorName, TEST_ROLE_NAME_1, KAFKA);
         // test: the conf is required argument
-        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1 };
-        SentryShellKafka sentryShell = new SentryShellKafka();
+        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1, "-t", "kafka" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + strOptionConf);
 
         // test: -r is required when create role
-        args = new String[] { "-cr", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-cr", "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -r is required when drop role
-        args = new String[] { "-dr", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-dr", "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -r is required when add role to group
-        args = new String[] { "-arg", "-g", "testGroup1", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-arg", "-g", "testGroup1", "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -g is required when add role to group
-        args = new String[] { "-arg", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-arg", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_GROUP_NAME);
 
         // test: -r is required when delete role from group
-        args = new String[] { "-drg", "-g", "testGroup1", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-drg", "-g", "testGroup1", "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -g is required when delete role from group
-        args = new String[] { "-drg", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-drg", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_GROUP_NAME);
 
         // test: -r is required when grant privilege to role
-        args = new String[] { "-gpr", "-p", "server=server1", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-gpr", "-p", "server=server1", "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -p is required when grant privilege to role
-        args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_PRIVILEGE);
 
         // test: action is required in privilege
-        args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-p", "host=*->topic=t1" };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-p", "host=*->topic=t1", "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
          try {
           getShellResultWithOSRedirect(sentryShell, args, false);
           fail("Expected IllegalArgumentException");
@@ -453,20 +453,20 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
          }
 
         // test: -r is required when revoke privilege from role
-        args = new String[] { "-rpr", "-p", "host=h1", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-rpr", "-p", "host=h1", "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -p is required when revoke privilege from role
-        args = new String[] { "-rpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] { "-rpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_PRIVILEGE);
 
         // test: command option is required for shell
-        args = new String[] {"-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellKafka();
+        args = new String[] {"-conf", confPath.getAbsolutePath(), "-t", "kafka" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsgsContains(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + "[",
                 "-arg Add role to group",
@@ -485,7 +485,7 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
   }
 
   // redirect the System.out to ByteArrayOutputStream, then execute the command and parse the result.
-  private Set<String> getShellResultWithOSRedirect(SentryShellKafka sentryShell,
+  private Set<String> getShellResultWithOSRedirect(SentryShellGeneric sentryShell,
       String[] args, boolean expectedExecuteResult) throws Exception {
     PrintStream oldOut = System.out;
     ByteArrayOutputStream outContent = new ByteArrayOutputStream();
@@ -512,13 +512,13 @@ public class TestSentryShellKafka extends SentryGenericServiceIntegrationBase {
     }
   }
 
-  private void validateMissingParameterMsg(SentryShellKafka sentryShell, String[] args,
+  private void validateMissingParameterMsg(SentryShellGeneric sentryShell, String[] args,
       String expectedErrorMsg) throws Exception {
     Set<String> errorMsgs = getShellResultWithOSRedirect(sentryShell, args, false);
     assertTrue("Expected error message: " + expectedErrorMsg, errorMsgs.contains(expectedErrorMsg));
   }
 
-  private void validateMissingParameterMsgsContains(SentryShellKafka sentryShell, String[] args,
+  private void validateMissingParameterMsgsContains(SentryShellGeneric sentryShell, String[] args,
       String ... expectedErrorMsgsContains) throws Exception {
     Set<String> errorMsgs = getShellResultWithOSRedirect(sentryShell, args, false);
     boolean foundAllMessages = false;

http://git-wip-us.apache.org/repos/asf/sentry/blob/d7a6667c/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 d4e26e8..d857107 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
@@ -80,32 +80,32 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
       @Override
       public void runTestAsSubject() throws Exception {
         // test: create role with -cr
-        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
         // test: create role with --create_role
         args = new String[] { "--create_role", "-r", TEST_ROLE_NAME_2, "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+            confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
 
         // validate the result, list roles with -lr
-        args = new String[] { "-lr", "-conf", confPath.getAbsolutePath() };
-        SentryShellSolr sentryShell = new SentryShellSolr();
+        args = new String[] { "-lr", "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         Set<String> roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1, TEST_ROLE_NAME_2);
 
         // validate the result, list roles with --list_role
-        args = new String[] { "--list_role", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "--list_role", "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1, TEST_ROLE_NAME_2);
 
         // test: drop role with -dr
-        args = new String[] { "-dr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+        args = new String[] { "-dr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
         // test: drop role with --drop_role
         args = new String[] { "--drop_role", "-r", TEST_ROLE_NAME_2, "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+            confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
 
         // validate the result
         Set<TSentryRole> roles = client.listAllRoles(requestorName, SOLR);
@@ -129,51 +129,51 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
         client.createRole(requestorName, TEST_ROLE_NAME_2, SOLR);
         // test: add role to group with -arg
         String[] args = { "-arg", "-r", TEST_ROLE_NAME_1, "-g", TEST_GROUP_1, "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+            confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
         // test: add role to multiple groups
         args = new String[] { "-arg", "-r", TEST_ROLE_NAME_1, "-g", TEST_GROUP_2 + "," + TEST_GROUP_3,
             "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+            confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
         // test: add role to group with --add_role_group
         args = new String[] { "--add_role_group", "-r", TEST_ROLE_NAME_2, "-g", TEST_GROUP_1,
             "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+            confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
 
         // validate the result list roles with -lr and -g
-        args = new String[] { "-lr", "-g", TEST_GROUP_1, "-conf", confPath.getAbsolutePath() };
-        SentryShellSolr sentryShell = new SentryShellSolr();
+        args = new String[] { "-lr", "-g", TEST_GROUP_1, "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         Set<String> roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1, TEST_ROLE_NAME_2);
 
         // list roles with --list_role and -g
         args = new String[] { "--list_role", "-g", TEST_GROUP_2, "-conf",
-            confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+            confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1);
 
         args = new String[] { "--list_role", "-g", TEST_GROUP_3, "-conf",
-            confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+            confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1);
 
         // test: delete role from group with -drg
         args = new String[] { "-drg", "-r", TEST_ROLE_NAME_1, "-g", TEST_GROUP_1, "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+            confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
         // test: delete role to multiple groups
         args = new String[] { "-drg", "-r", TEST_ROLE_NAME_1, "-g", TEST_GROUP_2 + "," + TEST_GROUP_3,
             "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+            confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
         // test: delete role from group with --delete_role_group
         args = new String[] { "--delete_role_group", "-r", TEST_ROLE_NAME_2, "-g", TEST_GROUP_1,
-            "-conf", confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+            "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
 
         // validate the result
         Set<TSentryRole> roles = client.listRolesByGroupName(requestorName, TEST_GROUP_1, SOLR);
@@ -199,17 +199,17 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
         client.createRole(requestorName, TEST_ROLE_NAME_1, SOLR);
         // add role to a group (lower case)
         String[] args = { "-arg", "-r", TEST_ROLE_NAME_1, "-g", "group1", "-conf",
-            confPath.getAbsolutePath() };
-        SentryShellSolr.main(args);
+            confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric.main(args);
 
         // validate the roles when group name is same case as above
-        args = new String[] { "-lr", "-g", "group1", "-conf", confPath.getAbsolutePath() };
-        SentryShellSolr sentryShell = new SentryShellSolr();
+        args = new String[] { "-lr", "-g", "group1", "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         Set<String> roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames, TEST_ROLE_NAME_1);
 
         // roles should be empty when group name is different case than above
-        args = new String[] { "-lr", "-g", "GROUP1", "-conf", confPath.getAbsolutePath() };
+        args = new String[] { "-lr", "-g", "GROUP1", "-conf", confPath.getAbsolutePath(), "-t", "solr" };
         roleNames = getShellResultWithOSRedirect(sentryShell, args, true);
         validateRoleNames(roleNames);
       }
@@ -245,13 +245,13 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
           // test: grant privilege to role
           String [] args = new String [] { grant(shortOption), "-r", TEST_ROLE_NAME_1, "-p",
             privs[ i ],
-            "-conf", confPath.getAbsolutePath() };
-          SentryShellSolr.main(args);
+            "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+          SentryShellGeneric.main(args);
         }
 
         // test the list privilege
-        String [] args = new String[] { list(shortOption), "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        SentryShellSolr sentryShell = new SentryShellSolr();
+        String [] args = new String[] { list(shortOption), "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         Set<String> privilegeStrs = getShellResultWithOSRedirect(sentryShell, args, true);
         assertEquals("Incorrect number of privileges", privs.length, privilegeStrs.size());
         for (int i = 0; i < privs.length; ++i) {
@@ -261,8 +261,8 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
         for (int i = 0; i < privs.length; ++i) {
           args = new String[] { revoke(shortOption), "-r", TEST_ROLE_NAME_1, "-p",
             privs[ i ], "-conf",
-            confPath.getAbsolutePath() };
-          SentryShellSolr.main(args);
+            confPath.getAbsolutePath(), "-t", "solr" };
+          SentryShellGeneric.main(args);
           Set<TSentryPrivilege> privileges = client.listPrivilegesByRoleName(requestorName,
             TEST_ROLE_NAME_1, SOLR, service);
           assertEquals("Incorrect number of privileges", privs.length - (i + 1), privileges.size());
@@ -294,8 +294,8 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
       public void runTestAsSubject() throws Exception {
         client.createRole(requestorName, TEST_ROLE_NAME_1, SOLR);
         // test: create duplicate role with -cr
-        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        SentryShellSolr sentryShell = new SentryShellSolr();
+        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for creating duplicate role");
@@ -304,8 +304,8 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
         }
 
         // test: drop non-exist role with -dr
-        args = new String[] { "-dr", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-dr", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for dropping non-exist role");
@@ -315,8 +315,8 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
 
         // test: add non-exist role to group with -arg
         args = new String[] { "-arg", "-r", TEST_ROLE_NAME_2, "-g", "testGroup1", "-conf",
-            confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+            confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for granting non-exist role to group");
@@ -326,8 +326,8 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
 
         // test: drop group from non-exist role with -drg
         args = new String[] { "-drg", "-r", TEST_ROLE_NAME_2, "-g", "testGroup1", "-conf",
-            confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+            confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for drop group from non-exist role");
@@ -337,8 +337,8 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
 
         // test: grant privilege to role with the error privilege format
         args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-p", "serverserver1->action=*",
-            "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+            "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for the error privilege format, invalid key value.");
@@ -349,8 +349,8 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
         // test: grant privilege to role with the error privilege hierarchy
         args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-p",
             "server=server1->table=tbl1->column=col2->action=insert", "-conf",
-            confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+            confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         try {
           sentryShell.executeShell(args);
           fail("Exception should be thrown for the error privilege format, invalid key value.");
@@ -372,62 +372,62 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
         String strOptionConf = "conf";
         client.createRole(requestorName, TEST_ROLE_NAME_1, SOLR);
         // test: the conf is required argument
-        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1 };
-        SentryShellSolr sentryShell = new SentryShellSolr();
+        String[] args = { "-cr", "-r", TEST_ROLE_NAME_1, "-t", "solr" };
+        SentryShellGeneric sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + strOptionConf);
 
         // test: -r is required when create role
-        args = new String[] { "-cr", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-cr", "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -r is required when drop role
-        args = new String[] { "-dr", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-dr", "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -r is required when add role to group
-        args = new String[] { "-arg", "-g", "testGroup1", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-arg", "-g", "testGroup1", "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -g is required when add role to group
-        args = new String[] { "-arg", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-arg", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_GROUP_NAME);
 
         // test: -r is required when delete role from group
-        args = new String[] { "-drg", "-g", "testGroup1", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-drg", "-g", "testGroup1", "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -g is required when delete role from group
-        args = new String[] { "-drg", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-drg", "-r", TEST_ROLE_NAME_2, "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_GROUP_NAME);
 
         // test: -r is required when grant privilege to role
-        args = new String[] { "-gpr", "-p", "server=server1", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-gpr", "-p", "server=server1", "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -p is required when grant privilege to role
-        args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_PRIVILEGE);
 
         // test: action is required in privilege
-        args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-p", "collection=collection1" };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-gpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-p", "collection=collection1", "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
          try {
           getShellResultWithOSRedirect(sentryShell, args, false);
           fail("Expected IllegalArgumentException");
@@ -436,20 +436,20 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
         }
 
         // test: -r is required when revoke privilege from role
-        args = new String[] { "-rpr", "-p", "server=server1", "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-rpr", "-p", "server=server1", "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_ROLE_NAME);
 
         // test: -p is required when revoke privilege from role
-        args = new String[] { "-rpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        args = new String[] { "-rpr", "-r", TEST_ROLE_NAME_1, "-conf", confPath.getAbsolutePath(), "-t", "solr" };
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsg(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + SentryShellCommon.OPTION_DESC_PRIVILEGE);
 
         // test: command option is required for shell
         args = new String[] {"-conf", confPath.getAbsolutePath() };
-        sentryShell = new SentryShellSolr();
+        sentryShell = new SentryShellGeneric();
         validateMissingParameterMsgsContains(sentryShell, args,
                 SentryShellCommon.PREFIX_MESSAGE_MISSING_OPTION + "[",
                 "-arg Add role to group",
@@ -468,7 +468,7 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
   }
 
   // redirect the System.out to ByteArrayOutputStream, then execute the command and parse the result.
-  private Set<String> getShellResultWithOSRedirect(SentryShellSolr sentryShell,
+  private Set<String> getShellResultWithOSRedirect(SentryShellGeneric sentryShell,
       String[] args, boolean expectedExecuteResult) throws Exception {
     PrintStream oldOut = System.out;
     ByteArrayOutputStream outContent = new ByteArrayOutputStream();
@@ -495,13 +495,13 @@ public class TestSentryShellSolr extends SentryGenericServiceIntegrationBase {
     }
   }
 
-  private void validateMissingParameterMsg(SentryShellSolr sentryShell, String[] args,
+  private void validateMissingParameterMsg(SentryShellGeneric sentryShell, String[] args,
       String expectedErrorMsg) throws Exception {
     Set<String> errorMsgs = getShellResultWithOSRedirect(sentryShell, args, false);
     assertTrue("Expected error message: " + expectedErrorMsg, errorMsgs.contains(expectedErrorMsg));
   }
 
-  private void validateMissingParameterMsgsContains(SentryShellSolr sentryShell, String[] args,
+  private void validateMissingParameterMsgsContains(SentryShellGeneric sentryShell, String[] args,
       String ... expectedErrorMsgsContains) throws Exception {
     Set<String> errorMsgs = getShellResultWithOSRedirect(sentryShell, args, false);
     boolean foundAllMessages = false;