You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by gc...@apache.org on 2016/04/14 07:15:43 UTC

sentry git commit: SENTRY-480: Create import tool that will load policy file about Solr into the DB store (Gregory Chanan, guoquanshen, reviewed by Vamsee Yarlagadda)

Repository: sentry
Updated Branches:
  refs/heads/master d540e3b5a -> 89aa9b4c1


SENTRY-480: Create import tool that will load policy file about Solr into the DB store (Gregory Chanan, guoquanshen, reviewed by Vamsee Yarlagadda)


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

Branch: refs/heads/master
Commit: 89aa9b4c18c652076bd26f4b7d73936dd94c2fe5
Parents: d540e3b
Author: Gregory Chanan <gc...@cloudera.com>
Authored: Tue Apr 5 18:18:27 2016 -0700
Committer: Gregory Chanan <gc...@cloudera.com>
Committed: Wed Apr 13 22:13:14 2016 -0700

----------------------------------------------------------------------
 sentry-provider/sentry-provider-db/pom.xml      |   1 -
 .../generic/tools/SentryConfigToolCommon.java   | 152 ++++++++++++
 .../db/generic/tools/SentryConfigToolSolr.java  | 246 ++++++++++++++++++
 .../tools/SolrTSentryPrivilegeConvertor.java    |  11 +-
 .../generic/tools/TestSentryConfigToolSolr.java | 248 +++++++++++++++++++
 .../src/test/resources/solr_case.ini            |  26 ++
 .../test/resources/solr_config_import_tool.ini  |  29 +++
 .../src/test/resources/solr_invalid.ini         |  21 ++
 8 files changed, 732 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/89aa9b4c/sentry-provider/sentry-provider-db/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/pom.xml b/sentry-provider/sentry-provider-db/pom.xml
index 8d9fa55..205ffe6 100644
--- a/sentry-provider/sentry-provider-db/pom.xml
+++ b/sentry-provider/sentry-provider-db/pom.xml
@@ -101,7 +101,6 @@ limitations under the License.
     <dependency>
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-provider-file</artifactId>
-      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.sentry</groupId>

http://git-wip-us.apache.org/repos/asf/sentry/blob/89aa9b4c/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryConfigToolCommon.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryConfigToolCommon.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryConfigToolCommon.java
new file mode 100644
index 0000000..013e824
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryConfigToolCommon.java
@@ -0,0 +1,152 @@
+/**
+ * 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 com.google.common.annotations.VisibleForTesting;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.Parser;
+
+abstract public class SentryConfigToolCommon {
+  private String policyFile;
+  private boolean validate;
+  private boolean importPolicy;
+  private boolean checkCompat;
+  private String confPath;
+
+ /**
+   *  parse arguments
+   * <pre>
+   *   -conf,--sentry_conf <filepath>     sentry config file path
+   *   -p,--policy_ini     <arg>          policy file path
+   *   -v,--validate                      validate policy file
+   *   -c,--checkcompat                   check compatibility with service
+   *   -i,--import                        import policy file
+   *   -h,--help                          print usage
+   * </pre>
+   * @param args
+   */
+  protected boolean parseArgs(String [] args) {
+    Options options = new Options();
+
+    Option globalPolicyPath = new Option("p", "policy_ini", true,
+        "Policy file path");
+    globalPolicyPath.setRequired(true);
+    options.addOption(globalPolicyPath);
+
+    Option validateOpt = new Option("v", "validate", false,
+        "Validate policy file");
+    validateOpt.setRequired(false);
+    options.addOption(validateOpt);
+
+    Option checkCompatOpt = new Option("c","checkcompat",false,
+        "Check compatibility with Sentry Service");
+    checkCompatOpt.setRequired(false);
+    options.addOption(checkCompatOpt);
+
+    Option importOpt = new Option("i", "import", false,
+        "Import policy file");
+    importOpt.setRequired(false);
+    options.addOption(importOpt);
+
+    // file path of sentry-site
+    Option sentrySitePathOpt = new Option("conf", "sentry_conf", true, "sentry-site file path");
+    sentrySitePathOpt.setRequired(true);
+    options.addOption(sentrySitePathOpt);
+
+    // help option
+    Option helpOpt = new Option("h", "help", false, "Shell usage");
+    helpOpt.setRequired(false);
+    options.addOption(helpOpt);
+
+    // this Options is parsed first for help option
+    Options helpOptions = new Options();
+    helpOptions.addOption(helpOpt);
+
+    try {
+      Parser parser = new GnuParser();
+
+      // parse help option first
+      CommandLine cmd = parser.parse(helpOptions, args, true);
+      for (Option opt : cmd.getOptions()) {
+        if (opt.getOpt().equals("h")) {
+          // get the help option, print the usage and exit
+          usage(options);
+          return false;
+        }
+      }
+
+      // without help option
+      cmd = parser.parse(options, args);
+
+      for (Option opt : cmd.getOptions()) {
+        if (opt.getOpt().equals("p")) {
+          policyFile = opt.getValue();
+        } else if (opt.getOpt().equals("v")) {
+          validate = true;
+        } else if (opt.getOpt().equals("i")) {
+          importPolicy = true;
+        } else if (opt.getOpt().equals("c")) {
+          checkCompat = true;
+        } else if (opt.getOpt().equals("conf")) {
+          confPath = opt.getValue();
+        }
+      }
+
+      if (!validate && !importPolicy) {
+        throw new IllegalArgumentException("No action specified; at least one of action or import must be specified");
+      }
+    } catch (ParseException pe) {
+      System.out.println(pe.getMessage());
+      usage(options);
+      return false;
+    }
+    return true;
+  }
+
+  // print usage
+  private void usage(Options sentryOptions) {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("sentryConfigTool", sentryOptions);
+  }
+
+  public abstract void run() throws Exception;
+
+  @VisibleForTesting
+  public boolean executeConfigTool(String [] args) throws Exception {
+    boolean result = true;
+    if (parseArgs(args)) {
+      run();
+    } else {
+      result = false;
+    }
+    return result;
+  }
+
+  public String getPolicyFile() { return policyFile; }
+  public boolean getValidate() { return validate; }
+  public boolean getImportPolicy() { return importPolicy; }
+  public boolean getCheckCompat() { return checkCompat; }
+  public String getConfPath() { return confPath; }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/89aa9b4c/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
new file mode 100644
index 0000000..bf91f52
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SentryConfigToolSolr.java
@@ -0,0 +1,246 @@
+/**
+ * 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 com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.common.collect.Table;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.sentry.core.common.Action;
+import org.apache.sentry.core.common.SentryConfigurationException;
+import org.apache.sentry.policy.common.KeyValue;
+import org.apache.sentry.policy.common.PolicyConstants;
+import org.apache.sentry.policy.search.SimpleSearchPolicyEngine;
+import org.apache.sentry.provider.common.ProviderBackend;
+import org.apache.sentry.provider.common.ProviderBackendContext;
+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.file.SimpleFileProviderBackend;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * SentryConfigToolSolr is an administrative tool used to parse a Solr policy file
+ * and add the role, group mappings, and privileges therein to the Sentry service.
+ */
+public class SentryConfigToolSolr extends SentryConfigToolCommon {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SentryConfigToolSolr.class);
+  public static final String SOLR_SERVICE_NAME = "sentry.service.client.solr.service.name";
+
+  @Override
+  public void run() throws Exception {
+    String component = "SOLR";
+    Configuration conf = getSentryConf();
+
+    String service = conf.get(SOLR_SERVICE_NAME, "service1");
+    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+    String requestorName = ugi.getShortUserName();
+
+    convertINIToSentryServiceCmds(component, service, requestorName,
+        conf, getPolicyFile(), getValidate(), getImportPolicy(), getCheckCompat());
+  }
+
+  private Configuration getSentryConf() {
+    Configuration conf = new Configuration();
+    conf.addResource(new Path(getConfPath()));
+    return conf;
+  }
+
+   /**
+    * Convert policy file to solrctl commands -- based on SENTRY-480
+    */
+  private void convertINIToSentryServiceCmds(String component,
+      String service, String requestorName, Configuration conf,
+      String policyFile, boolean validate, boolean importPolicy,
+      boolean checkCompat) throws Exception {
+
+    //instantiate a solr client for sentry service
+    SentryGenericServiceClient client = SentryGenericServiceClientFactory.create(conf);
+    //instantiate a file providerBackend for parsing
+    LOGGER.info("Reading policy file at: " + policyFile);
+    SimpleFileProviderBackend policyFileBackend =
+        new SimpleFileProviderBackend(conf, policyFile);
+    ProviderBackendContext context = new ProviderBackendContext();
+    context.setValidators(SimpleSearchPolicyEngine.createPrivilegeValidators());
+    policyFileBackend.initialize(context);
+    if (validate) {
+      validatePolicy(policyFileBackend);
+    }
+
+    if (checkCompat) {
+      checkCompat(policyFileBackend);
+    }
+
+    if (importPolicy) {
+      //import the relations about group,role and privilege into the DB store
+      Set<String> roles = Sets.newHashSet();
+      Table<String, String, Set<String>> groupRolePrivilegeTable =
+          policyFileBackend.getGroupRolePrivilegeTable();
+      SolrTSentryPrivilegeConvertor convertor = new SolrTSentryPrivilegeConvertor(component, service, false);
+
+      for (String groupName : groupRolePrivilegeTable.rowKeySet()) {
+        for (String roleName : groupRolePrivilegeTable.columnKeySet()) {
+          if (!roles.contains(roleName)) {
+            LOGGER.info("Creating role: " + roleName);
+            client.createRoleIfNotExist(requestorName, roleName, component);
+            roles.add(roleName);
+          }
+
+          Set<String> privileges = groupRolePrivilegeTable.get(groupName, roleName);
+          if (privileges == null) {
+            continue;
+          }
+          LOGGER.info("Adding role: " + roleName + " to group: " + groupName);
+          client.addRoleToGroups(requestorName, roleName, component, Sets.newHashSet(groupName));
+
+          for (String permission : privileges) {
+            String action = null;
+
+            for (String authorizable : PolicyConstants.AUTHORIZABLE_SPLITTER.
+                trimResults().split(permission)) {
+              KeyValue kv = new KeyValue(authorizable);
+              String key = kv.getKey();
+              String value = kv.getValue();
+              if ("action".equalsIgnoreCase(key)) {
+                action = value;
+              }
+            }
+
+            // Service doesn't support not specifying action
+            if (action == null) {
+              permission += "->action=" + Action.ALL;
+            }
+            LOGGER.info("Adding permission: " + permission + " to role: " + roleName);
+            client.grantPrivilege(requestorName, roleName, component, convertor.fromString(permission));
+          }
+        }
+      }
+    }
+  }
+
+  private void validatePolicy(ProviderBackend backend) throws Exception {
+    try {
+      backend.validatePolicy(true);
+    } catch (SentryConfigurationException e) {
+      printConfigErrorsWarnings(e);
+      throw e;
+    }
+  }
+
+  private void printConfigErrorsWarnings(SentryConfigurationException configException) {
+    System.out.println(" *** Found configuration problems *** ");
+    for (String errMsg : configException.getConfigErrors()) {
+      System.out.println("ERROR: " + errMsg);
+    }
+    for (String warnMsg : configException.getConfigWarnings()) {
+      System.out.println("Warning: " + warnMsg);
+    }
+  }
+
+  private void checkCompat(SimpleFileProviderBackend backend) throws Exception {
+    Map<String, Set<String>> rolesCaseMapping = new HashMap<String, Set<String>>();
+    Table<String, String, Set<String>> groupRolePrivilegeTable =
+      backend.getGroupRolePrivilegeTable();
+
+    for (String roleName : groupRolePrivilegeTable.columnKeySet()) {
+      String roleNameLower = roleName.toLowerCase(Locale.US);
+      if (!roleName.equals(roleNameLower)) {
+        if (!rolesCaseMapping.containsKey(roleNameLower)) {
+          rolesCaseMapping.put(roleNameLower, Sets.newHashSet(roleName));
+        } else {
+          rolesCaseMapping.get(roleNameLower).add(roleName);
+        }
+      }
+    }
+
+    List<String> errors = new LinkedList<String>();
+    StringBuilder warningString = new StringBuilder();
+    if (!rolesCaseMapping.isEmpty()) {
+      warningString.append("The following roles names will be lower cased when added to the Sentry Service.\n");
+      warningString.append("This will cause document-level security to fail to match the role tokens.\n");
+      warningString.append("Role names: ");
+    }
+    boolean firstWarning = true;
+
+    for (Map.Entry<String, Set<String>> entry : rolesCaseMapping.entrySet()) {
+      Set<String> caseMapping = entry.getValue();
+      if (caseMapping.size() > 1) {
+        StringBuilder errorString = new StringBuilder();
+        errorString.append("The following (cased) roles map to the same role in the sentry service: ");
+        boolean first = true;
+        for (String casedRole : caseMapping) {
+          errorString.append(first ? "" : ", ");
+          errorString.append(casedRole);
+          first = false;
+        }
+        errorString.append(".  Role in service: ").append(entry.getKey());
+        errors.add(errorString.toString());
+      }
+
+      for (String casedRole : caseMapping) {
+        warningString.append(firstWarning? "" : ", ");
+        warningString.append(casedRole);
+        firstWarning = false;
+      }
+    }
+
+    for (String error : errors) {
+      System.out.println("ERROR: " + error);
+    }
+    System.out.println("\n");
+
+    System.out.println("Warning: " + warningString.toString());
+    SentryConfigurationException ex =
+        new SentryConfigurationException("Compatibility check failure");
+    ex.setConfigErrors(errors);
+    ex.setConfigWarnings(Lists.<String>asList(warningString.toString(), new String[0]));
+    throw ex;
+  }
+
+  public static void main(String[] args) throws Exception {
+    SentryConfigToolSolr solrTool = new SentryConfigToolSolr();
+    try {
+      solrTool.executeConfigTool(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/89aa9b4c/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SolrTSentryPrivilegeConvertor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SolrTSentryPrivilegeConvertor.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SolrTSentryPrivilegeConvertor.java
index e2dfdf1..8dffe94 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SolrTSentryPrivilegeConvertor.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/tools/SolrTSentryPrivilegeConvertor.java
@@ -42,14 +42,23 @@ import java.util.List;
 public  class SolrTSentryPrivilegeConvertor implements TSentryPrivilegeConvertor {
   private String component;
   private String service;
+  private boolean validate;
 
   public SolrTSentryPrivilegeConvertor(String component, String service) {
+    this(component, service, true);
+  }
+
+  public SolrTSentryPrivilegeConvertor(String component, String service, boolean validate) {
     this.component = component;
     this.service = service;
+    this.validate = validate;
   }
 
   public TSentryPrivilege fromString(String privilegeStr) throws Exception {
-    validatePrivilegeHierarchy(privilegeStr);
+    if (validate) {
+      validatePrivilegeHierarchy(privilegeStr);
+    }
+
     TSentryPrivilege tSentryPrivilege = new TSentryPrivilege();
     List<TAuthorizable> authorizables = new LinkedList<TAuthorizable>();
     for (String authorizable : PolicyConstants.AUTHORIZABLE_SPLITTER.split(privilegeStr)) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/89aa9b4c/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
new file mode 100644
index 0000000..7149f17
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/tools/TestSentryConfigToolSolr.java
@@ -0,0 +1,248 @@
+/**
+ * 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 com.google.common.io.Files;
+import com.google.common.collect.Sets;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceIntegrationBase;
+import org.apache.sentry.provider.db.generic.service.thrift.TSentryRole;
+import org.apache.sentry.provider.db.generic.service.thrift.TSentryPrivilege;
+import org.apache.sentry.core.common.SentryConfigurationException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestSentryConfigToolSolr extends SentryGenericServiceIntegrationBase {
+  private static String RESOURCES_DIR = "target" + File.separator + "test-classes" + File.separator;
+  private static String VALID_POLICY_INI = RESOURCES_DIR + "solr_config_import_tool.ini";
+  private static String INVALID_POLICY_INI = RESOURCES_DIR + "solr_invalid.ini";
+  private static String CASE_POLICY_INI = RESOURCES_DIR + "solr_case.ini";
+  private File confDir;
+  private File confPath;
+  private String requestorName = "";
+  private String service = "service1";
+
+  @Before
+  public void prepareForTest() throws Exception {
+    confDir = Files.createTempDir();
+    confPath = new File(confDir, "sentry-site.xml");
+    if (confPath.createNewFile()) {
+      FileOutputStream to = new FileOutputStream(confPath);
+      conf.writeXml(to);
+      to.close();
+    }
+    requestorName = System.getProperty("user.name", "");
+    Set<String> requestorUserGroupNames = Sets.newHashSet(ADMIN_GROUP);
+    setLocalGroupMapping(requestorName, requestorUserGroupNames);
+    // add ADMIN_USER for the after() in SentryServiceIntegrationBase
+    setLocalGroupMapping(ADMIN_USER, requestorUserGroupNames);
+    writePolicyFile();
+  }
+
+  @After
+  public void clearTestData() throws Exception {
+    FileUtils.deleteQuietly(confDir);
+
+    // clear roles and privileges
+    Set<TSentryRole> tRoles = client.listAllRoles(requestorName, SOLR);
+    for (TSentryRole tRole : tRoles) {
+      String role = tRole.getRoleName();
+      Set<TSentryPrivilege> privileges = client.listPrivilegesByRoleName(
+          requestorName, role, SOLR, service);
+      for (TSentryPrivilege privilege : privileges) {
+        client.revokePrivilege(requestorName, role, SOLR, privilege);
+      }
+      client.dropRole(requestorName, role, SOLR);
+    }
+  }
+
+  @Test
+  public void testConvertIni() throws Exception {
+    runTestAsSubject(new TestOperation() {
+      @Override
+      public void runTestAsSubject() throws Exception {
+        String[] args = { "-p", VALID_POLICY_INI, "-conf", confPath.getAbsolutePath(), "-v", "-i"};
+        SentryConfigToolSolr sentryTool = new SentryConfigToolSolr();
+        sentryTool.executeConfigTool(args);
+
+        Map<String, Set<String>> groupMapping = new HashMap<String, Set<String>>();
+        groupMapping.put("corporal_role", Sets.newHashSet("corporal", "sergeant", "general", "commander_in_chief"));
+        groupMapping.put("sergeant_role", Sets.newHashSet("sergeant", "general", "commander_in_chief"));
+        groupMapping.put("general_role", Sets.newHashSet("general", "commander_in_chief"));
+        groupMapping.put("commander_in_chief_role", Sets.newHashSet("commander_in_chief"));
+
+        Map<String, Set<String>> privilegeMapping = new HashMap<String, Set<String>>();
+        privilegeMapping.put("corporal_role",
+            Sets.newHashSet("Collection=info->action=query", "Collection=info->action=update"));
+        privilegeMapping.put("sergeant_role",
+            Sets.newHashSet("Collection=info->action=update"));
+        privilegeMapping.put("general_role",
+            Sets.newHashSet("Collection=info->action=*"));
+        privilegeMapping.put("commander_in_chief_role",
+            Sets.newHashSet("Collection=*->action=*"));
+
+        // check roles
+        Set<TSentryRole> tRoles = client.listAllRoles(requestorName, SOLR);
+        assertEquals("Unexpected number of roles", groupMapping.keySet().size(), tRoles.size());
+        Set<String> roles = new HashSet<String>();
+        for (TSentryRole tRole : tRoles) {
+          roles.add(tRole.getRoleName());
+        }
+
+        for (String expectedRole : groupMapping.keySet()) {
+          assertTrue("Didn't find expected role: " + expectedRole, roles.contains(expectedRole));
+        }
+
+        // check groups
+        for (TSentryRole tRole : tRoles) {
+          Set<String> expectedGroups = groupMapping.get(tRole.getRoleName());
+          assertEquals("Group size doesn't match for role: " + tRole.getRoleName(),
+              expectedGroups.size(), tRole.getGroups().size());
+          assertTrue("Group does not contain all expected members for role: " + tRole.getRoleName(),
+              tRole.getGroups().containsAll(expectedGroups));
+        }
+
+        // check privileges
+        SolrTSentryPrivilegeConvertor convert = new SolrTSentryPrivilegeConvertor(SOLR, service);
+        for (String role : roles) {
+          Set<TSentryPrivilege> privileges = client.listPrivilegesByRoleName(
+              requestorName, role, SOLR, service);
+          Set<String> expectedPrivileges = privilegeMapping.get(role);
+          assertEquals("Privilege set size doesn't match for role: " + role,
+              expectedPrivileges.size(), privileges.size());
+
+          Set<String> privilegeStrs = new HashSet<String>();
+          for (TSentryPrivilege privilege : privileges) {
+            privilegeStrs.add(convert.toString(privilege));
+          }
+
+          for (String expectedPrivilege : expectedPrivileges) {
+            assertTrue("Did not find expected privilege: " + expectedPrivilege,
+                privilegeStrs.contains(expectedPrivilege));
+          }
+        }
+      }
+    });
+  }
+
+  @Test
+  public void testNoPolicyFile() throws Exception {
+    runTestAsSubject(new TestOperation() {
+      @Override
+      public void runTestAsSubject() throws Exception {
+        String[] args = { "-p", INVALID_POLICY_INI + "Foobar", "-conf", confPath.getAbsolutePath(), "-v", "-i"};
+        SentryConfigToolSolr sentryTool = new SentryConfigToolSolr();
+        try {
+          sentryTool.executeConfigTool(args);
+          fail("Exception should be thrown for nonexistant ini");
+        } catch (SentryConfigurationException e) {
+          // expected exception
+        }
+      }
+    });
+  }
+
+  @Test
+  public void testNoValidateNorImport() throws Exception {
+    runTestAsSubject(new TestOperation() {
+      @Override
+      public void runTestAsSubject() throws Exception {
+        String[] args = { "-p", INVALID_POLICY_INI, "-conf", confPath.getAbsolutePath()};
+        SentryConfigToolSolr sentryTool = new SentryConfigToolSolr();
+        try {
+          sentryTool.executeConfigTool(args);
+          fail("Exception should be thrown for validating invalid ini");
+        } catch (IllegalArgumentException e) {
+          // expected exception
+        }
+      }
+    });
+  }
+
+  @Test
+  public void testConvertInvalidIni() throws Exception {
+    runTestAsSubject(new TestOperation() {
+      @Override
+      public void runTestAsSubject() throws Exception {
+        // test: validate an invalid ini
+        String[] args = { "-p", INVALID_POLICY_INI, "-conf", confPath.getAbsolutePath(), "-v", "-i"};
+        SentryConfigToolSolr sentryTool = new SentryConfigToolSolr();
+        try {
+          sentryTool.executeConfigTool(args);
+          fail("Exception should be thrown for validating invalid ini");
+        } catch (SentryConfigurationException e) {
+          // expected exception
+        }
+
+        // test without validating, should not error
+        args = new String[] { "-p", INVALID_POLICY_INI, "-conf", confPath.getAbsolutePath(), "-i"};
+        sentryTool = new SentryConfigToolSolr();
+        sentryTool.executeConfigTool(args);
+      }
+    });
+  }
+
+  @Test
+  public void testCompatCheck() throws Exception {
+    runTestAsSubject(new TestOperation() {
+      @Override
+      public void runTestAsSubject() throws Exception {
+        // test: validate an invalid ini
+        String[] args = { "-p", CASE_POLICY_INI, "-conf", confPath.getAbsolutePath(), "-v", "-i", "-c"};
+        SentryConfigToolSolr sentryTool = new SentryConfigToolSolr();
+        try {
+          sentryTool.executeConfigTool(args);
+          fail("Exception should be thrown for validating invalid ini");
+        } catch (SentryConfigurationException e) {
+          assertEquals("Expected error", 1, e.getConfigErrors().size());
+          String error = e.getConfigErrors().get(0);
+          assertCasedRoleNamesInMessage(error, "RoLe1", "rOlE1");
+          String warning = e.getConfigWarnings().get(0);
+          assertCasedRoleNamesInMessage(warning, "ROLE2", "RoLe1", "rOlE1");
+          assertEquals("Expected warning", 1, e.getConfigWarnings().size());
+        }
+
+        // test without compat checking
+        args = new String[] { "-p", CASE_POLICY_INI, "-conf", confPath.getAbsolutePath(), "-i", "-v"};
+        sentryTool = new SentryConfigToolSolr();
+        sentryTool.executeConfigTool(args);
+      }
+    });
+  }
+
+  private void assertCasedRoleNamesInMessage(String message, String ... casedRoleNames) {
+    for (String casedRoleName : casedRoleNames) {
+      assertTrue("Expected cased role name: " + casedRoleName, message.contains(casedRoleName));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/89aa9b4c/sentry-provider/sentry-provider-db/src/test/resources/solr_case.ini
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/resources/solr_case.ini b/sentry-provider/sentry-provider-db/src/test/resources/solr_case.ini
new file mode 100644
index 0000000..fbbebfc
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/test/resources/solr_case.ini
@@ -0,0 +1,26 @@
+# 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.
+
+[groups]
+groupa = RoLe1
+groupb = rOlE1
+groupc = ROLE2
+
+[roles]
+RoLe1 = collection=*
+rOlE1 = collection=*
+ROLE2 = collection=*

http://git-wip-us.apache.org/repos/asf/sentry/blob/89aa9b4c/sentry-provider/sentry-provider-db/src/test/resources/solr_config_import_tool.ini
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/resources/solr_config_import_tool.ini b/sentry-provider/sentry-provider-db/src/test/resources/solr_config_import_tool.ini
new file mode 100644
index 0000000..da7df4c
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/test/resources/solr_config_import_tool.ini
@@ -0,0 +1,29 @@
+# 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.
+
+[groups]
+corporal = corporal_role
+sergeant = corporal_role, sergeant_role
+general = corporal_role, sergeant_role, general_role
+commander_in_chief = corporal_role, sergeant_role, general_role, commander_in_chief_role
+
+[roles]
+corporal_role = collection=info->action=query, \
+  collection=info->action=update
+sergeant_role = collection=info->action=update
+general_role = collection=info->action=*
+commander_in_chief_role = collection=*

http://git-wip-us.apache.org/repos/asf/sentry/blob/89aa9b4c/sentry-provider/sentry-provider-db/src/test/resources/solr_invalid.ini
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/resources/solr_invalid.ini b/sentry-provider/sentry-provider-db/src/test/resources/solr_invalid.ini
new file mode 100644
index 0000000..03083a7
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/test/resources/solr_invalid.ini
@@ -0,0 +1,21 @@
+# 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.
+
+[groups]
+
+[roles]
+