You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sd...@apache.org on 2015/08/14 09:29:11 UTC

[37/50] [abbrv] incubator-sentry git commit: SENTRY-197: Create tool to dump and load of entire Sentry service (Colin Ma, Reviewed by:Sravya Tirukkovalur, Guoquan Shen, Dapeng Sun, Anne Yu)

SENTRY-197: Create tool to dump and load of entire Sentry service (Colin Ma, Reviewed by:Sravya Tirukkovalur, Guoquan Shen, Dapeng Sun, Anne Yu)


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

Branch: refs/heads/hive_plugin_v2
Commit: 92cde111f232a98bbce4b320100d408668cc444c
Parents: 6c3184a
Author: Colin Ma <co...@apache.org>
Authored: Thu Jul 30 08:38:57 2015 +0800
Committer: Colin Ma <co...@apache.org>
Committed: Thu Jul 30 08:38:57 2015 +0800

----------------------------------------------------------------------
 .../hive/SentryIniPolicyFileFormatter.java      |  161 ++
 .../hive/SentryPolicyFileFormatFactory.java     |   44 +
 .../binding/hive/SentryPolicyFileFormatter.java |   39 +
 .../binding/hive/authz/SentryConfigTool.java    |  234 +--
 .../sentry/binding/hive/conf/HiveAuthzConf.java |   14 +-
 .../hive/TestSentryIniPolicyFileFormatter.java  |  220 +++
 .../db/service/thrift/SentryPolicyService.java  | 1612 ++++++++++++++++++
 .../thrift/TSentryExportMappingDataRequest.java |  486 ++++++
 .../TSentryExportMappingDataResponse.java       |  496 ++++++
 .../thrift/TSentryImportMappingDataRequest.java |  689 ++++++++
 .../TSentryImportMappingDataResponse.java       |  390 +++++
 .../db/service/thrift/TSentryMappingData.java   |  695 ++++++++
 .../db/service/persistent/SentryStore.java      |  458 ++++-
 .../thrift/SentryPolicyServiceClient.java       |    8 +
 .../SentryPolicyServiceClientDefaultImpl.java   |  111 ++
 .../thrift/SentryPolicyStoreProcessor.java      |   53 +
 .../service/thrift/SentryServiceUtil.java       |  127 ++
 .../main/resources/sentry_policy_service.thrift |   41 +-
 .../persistent/TestSentryStoreImportExport.java |  899 ++++++++++
 .../thrift/TestSentryServiceImportExport.java   |  538 ++++++
 .../sentry/tests/e2e/hive/TestPolicyImport.java |  199 ---
 .../tests/e2e/hive/TestPolicyImportExport.java  |  195 +++
 .../src/test/resources/testPolicyImport.ini     |   25 +
 .../test/resources/testPolicyImportAdmin.ini    |   22 +
 .../test/resources/testPolicyImportError.ini    |   21 +
 25 files changed, 7362 insertions(+), 415 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92cde111/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryIniPolicyFileFormatter.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryIniPolicyFileFormatter.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryIniPolicyFileFormatter.java
new file mode 100644
index 0000000..79164da
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryIniPolicyFileFormatter.java
@@ -0,0 +1,161 @@
+/**
+ * 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.binding.hive;
+
+import java.io.File;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.sentry.provider.common.PolicyFileConstants;
+import org.apache.sentry.provider.common.ProviderBackendContext;
+import org.apache.sentry.provider.common.ProviderConstants;
+import org.apache.sentry.provider.file.SimpleFileProviderBackend;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.common.collect.Table;
+import com.google.common.io.Files;
+
+/**
+ * SentryIniPolicyFileFormatter is to parse file and write data to file for sentry mapping data with
+ * ini format, eg:
+ * [groups]
+ * group1=role1
+ * [roles]
+ * role1=server=server1
+ */
+public class SentryIniPolicyFileFormatter implements SentryPolicyFileFormatter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SentryIniPolicyFileFormatter.class);
+
+  private static final String NL = System.getProperty("line.separator", "\n");
+
+  /**
+   * Write the sentry mapping data to ini file.
+   * 
+   * @param resourcePath
+   *        The path of the output file
+   * @param sentryMappingData
+   *        The map for sentry mapping data, eg:
+   *        for the following mapping data:
+   *        group1=role1,role2
+   *        group2=role2,role3
+   *        role1=server=server1->db=db1
+   *        role2=server=server1->db=db1->table=tbl1,server=server1->db=db1->table=tbl2
+   *        role3=server=server1->url=hdfs://localhost/path
+   * 
+   *        The sentryMappingData will be inputed as:
+   *        {
+   *        groups={[group1={role1, role2}], group2=[role2, role3]},
+   *        roles={role1=[server=server1->db=db1],
+   *        role2=[server=server1->db=db1->table=tbl1,server=server1->db=db1->table=tbl2],
+   *        role3=[server=server1->url=hdfs://localhost/path]
+   *        }
+   *        }
+   */
+  @Override
+  public void write(String resourcePath, Map<String, Map<String, Set<String>>> sentryMappingData)
+      throws Exception {
+    File destFile = new File(resourcePath);
+    if (destFile.exists() && !destFile.delete()) {
+      throw new IllegalStateException("Unable to delete " + destFile);
+    }
+    String contents = Joiner
+        .on(NL)
+        .join(
+        generateSection(PolicyFileConstants.GROUPS,
+                sentryMappingData.get(PolicyFileConstants.GROUPS)),
+        generateSection(PolicyFileConstants.ROLES,
+                sentryMappingData.get(PolicyFileConstants.ROLES)),
+            "");
+    LOGGER.info("Writing policy file to " + destFile + ":\n" + contents);
+    Files.write(contents, destFile, Charsets.UTF_8);
+  }
+
+  /**
+   * parse the ini file and return a map with all data
+   * 
+   * @param resourcePath
+   *        The path of the input file
+   * @param conf
+   *        The configuration info
+   * @return the result of sentry mapping data in map structure.
+   */
+  @Override
+  public Map<String, Map<String, Set<String>>> parse(String resourcePath, Configuration conf)
+      throws Exception {
+    Map<String, Map<String, Set<String>>> resultMap = Maps.newHashMap();
+    // SimpleFileProviderBackend is used for parse the ini file
+    SimpleFileProviderBackend policyFileBackend = new SimpleFileProviderBackend(conf, resourcePath);
+    ProviderBackendContext context = new ProviderBackendContext();
+    context.setAllowPerDatabase(true);
+    // parse the ini file
+    policyFileBackend.initialize(context);
+
+    // SimpleFileProviderBackend parsed the input file and output the data in Table format.
+    Table<String, String, Set<String>> groupRolePrivilegeTable = policyFileBackend
+        .getGroupRolePrivilegeTable();
+    Map<String, Set<String>> groupRolesMap = Maps.newHashMap();
+    Map<String, Set<String>> rolePrivilegesMap = Maps.newHashMap();
+    for (String groupName : groupRolePrivilegeTable.rowKeySet()) {
+      for (String roleName : groupRolePrivilegeTable.columnKeySet()) {
+        // get the roles set for the current groupName
+        Set<String> tempRoles = groupRolesMap.get(groupName);
+        if (tempRoles == null) {
+          tempRoles = Sets.newHashSet();
+        }
+        Set<String> privileges = groupRolePrivilegeTable.get(groupName, roleName);
+        // if there has privilege for [group,role], if no privilege exist, the [group, role] info
+        // will be discard.
+        if (privileges != null) {
+          // update [group, role] mapping data
+          tempRoles.add(roleName);
+          groupRolesMap.put(groupName, tempRoles);
+          // update [role, privilege] mapping data
+          rolePrivilegesMap.put(roleName, privileges);
+        }
+      }
+    }
+    resultMap.put(PolicyFileConstants.GROUPS, groupRolesMap);
+    resultMap.put(PolicyFileConstants.ROLES, rolePrivilegesMap);
+    return resultMap;
+  }
+
+  // generate the ini section according to the mapping data.
+  private String generateSection(String name, Map<String, Set<String>> mappingData) {
+    if (mappingData.isEmpty()) {
+      return "";
+    }
+    List<String> lines = Lists.newArrayList();
+    lines.add("[" + name + "]");
+    for (String key : mappingData.keySet()) {
+      lines.add(ProviderConstants.KV_JOINER.join(key,
+          ProviderConstants.ROLE_JOINER.join(mappingData.get(key))));
+    }
+    return Joiner.on(NL).join(lines);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92cde111/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryPolicyFileFormatFactory.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryPolicyFileFormatFactory.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryPolicyFileFormatFactory.java
new file mode 100644
index 0000000..d2c6072
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryPolicyFileFormatFactory.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sentry.binding.hive;
+
+import java.lang.reflect.Constructor;
+
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
+
+/**
+ * SentryPolicyFileFormatFactory is used to create FileFormatter for different file type according
+ * to the configuration, the default FileFormatter is for ini file.
+ */
+public class SentryPolicyFileFormatFactory {
+
+  public static SentryPolicyFileFormatter createFileFormatter(HiveAuthzConf conf) throws Exception {
+    // The default formatter is org.apache.sentry.binding.hive.SentryIniPolicyFileFormatter, for ini
+    // file.
+    String policyFileFormatterName = conf.get(AuthzConfVars.AUTHZ_POLICY_FILE_FORMATTER.getVar());
+    // load the policy file formatter class
+    Constructor<?> policyFileFormatterConstructor = Class.forName(policyFileFormatterName)
+        .getDeclaredConstructor();
+    policyFileFormatterConstructor.setAccessible(true);
+    SentryPolicyFileFormatter sentryPolicyFileFormatter = (SentryPolicyFileFormatter) policyFileFormatterConstructor
+        .newInstance();
+    return sentryPolicyFileFormatter;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92cde111/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryPolicyFileFormatter.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryPolicyFileFormatter.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryPolicyFileFormatter.java
new file mode 100644
index 0000000..14437ca
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryPolicyFileFormatter.java
@@ -0,0 +1,39 @@
+/**
+ * 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.binding.hive;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * SentryPolicyFileFormatter is to parse file and write data to file for sentry mapping data.
+ */
+public interface SentryPolicyFileFormatter {
+
+  // write the sentry mapping data to file
+  public void write(String resourcePath, Map<String, Map<String, Set<String>>> sentryMappingData)
+      throws Exception;
+
+  // parse the sentry mapping data from file
+  public Map<String, Map<String, Set<String>>> parse(String resourcePath, Configuration conf)
+      throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92cde111/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
index 4388ca0..d9bb42d 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
@@ -23,7 +23,7 @@ import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.cli.CommandLine;
@@ -34,6 +34,7 @@ import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.Parser;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.Driver;
@@ -45,25 +46,28 @@ import org.apache.log4j.LogManager;
 import org.apache.sentry.Command;
 import org.apache.sentry.binding.hive.HiveAuthzBindingHook;
 import org.apache.sentry.binding.hive.HiveAuthzBindingSessionHook;
+import org.apache.sentry.binding.hive.SentryPolicyFileFormatFactory;
+import org.apache.sentry.binding.hive.SentryPolicyFileFormatter;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
 import org.apache.sentry.core.common.SentryConfigurationException;
 import org.apache.sentry.core.common.Subject;
-import org.apache.sentry.core.model.db.AccessConstants;
-import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.Server;
-import org.apache.sentry.policy.db.DBModelAuthorizables;
 import org.apache.sentry.provider.common.AuthorizationProvider;
-import org.apache.sentry.provider.common.KeyValue;
-import org.apache.sentry.provider.common.ProviderBackendContext;
-import org.apache.sentry.provider.common.ProviderConstants;
 import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-import org.apache.sentry.provider.db.service.thrift.TSentryRole;
-import org.apache.sentry.provider.file.SimpleFileProviderBackend;
 import org.apache.sentry.service.thrift.SentryServiceClientFactory;
 
-import com.google.common.collect.Table;
-
+/**
+ * set the required system property to be read by HiveConf and AuthzConf
+ * 
+ * @throws Exception
+ */
+// Hack, hiveConf doesn't provide a reliable way check if it found a valid
+// hive-site
+// load auth provider
+// get the configured sentry provider
+// validate policy files
+// import policy files
 public class SentryConfigTool {
   private String sentrySiteFile = null;
   private String policyFile = null;
@@ -71,9 +75,11 @@ public class SentryConfigTool {
   private String jdbcURL = null;
   private String user = null;
   private String passWord = null;
+  private String importPolicyFilePath = null;
+  private String exportPolicyFilePath = null;
   private boolean listPrivs = false;
   private boolean validate = false;
-  private boolean importPolicy = false;
+  private boolean importOverwriteRole = false;
   private HiveConf hiveConf = null;
   private HiveAuthzConf authzConf = null;
   private AuthorizationProvider sentryProvider = null;
@@ -114,12 +120,20 @@ public class SentryConfigTool {
     this.validate = validate;
   }
 
-  public boolean isImportPolicy() {
-    return importPolicy;
+  public String getImportPolicyFilePath() {
+    return importPolicyFilePath;
+  }
+
+  public void setImportPolicyFilePath(String importPolicyFilePath) {
+    this.importPolicyFilePath = importPolicyFilePath;
   }
 
-  public void setImportPolicy(boolean importPolicy) {
-    this.importPolicy = importPolicy;
+  public String getExportPolicyFilePath() {
+    return exportPolicyFilePath;
+  }
+
+  public void setExportPolicyFilePath(String exportPolicyFilePath) {
+    this.exportPolicyFilePath = exportPolicyFilePath;
   }
 
   public String getSentrySiteFile() {
@@ -178,6 +192,14 @@ public class SentryConfigTool {
     this.listPrivs = listPrivs;
   }
 
+  public boolean isImportOverwriteRole() {
+    return importOverwriteRole;
+  }
+
+  public void setImportOverwriteRole(boolean importOverwriteRole) {
+    this.importOverwriteRole = importOverwriteRole;
+  }
+
   /**
    * set the required system property to be read by HiveConf and AuthzConf
    * @throws Exception
@@ -251,133 +273,33 @@ public class SentryConfigTool {
     System.out.println("No errors found in the policy file");
   }
 
-  // import policy files
+  // import the sentry mapping data to database
   public void importPolicy() throws Exception {
-    final String requestorUserName = "hive";
-    SimpleFileProviderBackend policyFileBackend;
-    SentryPolicyServiceClient client;
-
-    policyFileBackend = new SimpleFileProviderBackend(getAuthzConf(),
-        getAuthzConf().get(AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getVar()));
-    ProviderBackendContext context = new ProviderBackendContext();
-    context.setAllowPerDatabase(true);
-    policyFileBackend.initialize(context);
-    client = SentryServiceClientFactory.create(getAuthzConf());
-    Set<String> roles = new HashSet<String>();
-    for (TSentryRole sentryRole : client.listRoles(requestorUserName)) {
-      roles.add(sentryRole.getRoleName());
-    }
-
-    Table<String, String, Set<String>> groupRolePrivilegeTable =
-        policyFileBackend.getGroupRolePrivilegeTable();
-    for(String groupName : groupRolePrivilegeTable.rowKeySet()) {
-      for(String roleName : groupRolePrivilegeTable.columnKeySet()) {
-        if (!roles.contains(roleName)) {
-          client.createRole(requestorUserName, roleName);
-          System.out.println(String.format("CREATE ROLE %s;", roleName));
-          roles.add(roleName);
-        }
-
-        Set<String> privileges = groupRolePrivilegeTable.get(groupName, roleName);
-        if (privileges == null) {
-          continue;
-        }
-        client.grantRoleToGroup(requestorUserName, groupName, roleName);
-        System.out.println(String.format("GRANT ROLE %s TO GROUP %s;",
-            roleName, groupName));
-
-        for (String permission : privileges) {
-          String server = null;
-          String database = null;
-          String table = null;
-          String column = null;
-          String uri = null;
-          String action = AccessConstants.ALL;
-          for (String authorizable : ProviderConstants.AUTHORIZABLE_SPLITTER.
-              trimResults().split(permission)) {
-            KeyValue kv = new KeyValue(authorizable);
-            DBModelAuthorizable a = DBModelAuthorizables.from(kv);
-            if (a == null) {
-              action = kv.getValue();
-              continue;
-            }
-
-            switch (a.getAuthzType()) {
-              case Server:
-                server = a.getName();
-                break;
-              case Db:
-                database = a.getName();
-                break;
-              case Table:
-              case View:
-                table = a.getName();
-                break;
-              case URI:
-                uri = a.getName();
-                break;
-              case Column:
-                column = a.getName();
-                break;
-              default:
-                break;
-            }
-          }
-
-          if (uri != null) {
-            System.out.println(String.format(
-                "# server=%s",
-                server));
-            System.out.println(String.format(
-                "GRANT ALL ON URI %s TO ROLE %s;",
-                uri, roleName));
-
-            client.grantURIPrivilege(requestorUserName, roleName, server, uri);
-          } else if (column != null && !AccessConstants.ALL.equals(column)) {
-            System.out.println(String.format(
-                "# server=%s, database=%s",
-                server, database));
-            System.out.println(String.format(
-                "GRANT %s (%s) ON TABLE %s TO ROLE %s;",
-                "*".equals(action) ? "ALL" : action.toUpperCase(), column,
-                table, roleName));
-
-            client.grantColumnPrivilege(requestorUserName, roleName, server,
-                database, table, column, action);
-          } else if (table != null && !AccessConstants.ALL.equals(table)) {
-            System.out.println(String.format(
-                "# server=%s, database=%s",
-                server, database));
-            System.out.println(String.format(
-                "GRANT %s ON TABLE %s TO ROLE %s;",
-                "*".equals(action) ? "ALL" : action.toUpperCase(), table,
-                roleName));
-
-            client.grantTablePrivilege(requestorUserName, roleName, server,
-                database, table, action);
-          } else if (database != null && !AccessConstants.ALL.equals(database)) {
-            System.out.println(String.format(
-                "# server=%s",
-                server));
-            System.out.println(String.format(
-                "GRANT %s ON DATABASE %s TO ROLE %s;",
-                "*".equals(action) ? "ALL" : action.toUpperCase(),
-                database, roleName));
-
-            client.grantDatabasePrivilege(requestorUserName, roleName, server,
-                database, action);
-          } else if (server != null) {
-            System.out.println(String.format("GRANT ALL ON SERVER %s TO ROLE %s;",
-                server, roleName));
-
-            client.grantServerPrivilege(requestorUserName, roleName, server, action);
-          } else {
-            System.out.println(String.format("No grant for permission %s",
-                permission));
-          }
-        }
-      }
-    }
+    String requestorUserName = System.getProperty("user.name", "");
+    // get the FileFormatter according to the configuration
+    SentryPolicyFileFormatter sentryPolicyFileFormatter = SentryPolicyFileFormatFactory
+        .createFileFormatter(authzConf);
+    // parse the input file, get the mapping data in map structure
+    Map<String, Map<String, Set<String>>> policyFileMappingData = sentryPolicyFileFormatter.parse(
+        importPolicyFilePath, authzConf);
+    // todo: here should be an validator to check the data's value, format, hierarchy
+    SentryPolicyServiceClient client = SentryServiceClientFactory.create(getAuthzConf());
+    // import the mapping data to database
+    client.importPolicy(policyFileMappingData, requestorUserName, importOverwriteRole);
+  }
+
+  // export the sentry mapping data to file
+  public void exportPolicy() throws Exception {
+    String requestorUserName = System.getProperty("user.name", "");
+    SentryPolicyServiceClient client = SentryServiceClientFactory.create(getAuthzConf());
+    // export the sentry mapping data from database to map structure
+    Map<String, Map<String, Set<String>>> policyFileMappingData = client
+        .exportPolicy(requestorUserName);
+    // get the FileFormatter according to the configuration
+    SentryPolicyFileFormatter sentryPolicyFileFormatter = SentryPolicyFileFormatFactory
+        .createFileFormatter(authzConf);
+    // write the sentry mapping data to exportPolicyFilePath with the data in map structure
+    sentryPolicyFileFormatter.write(exportPolicyFilePath, policyFileMappingData);
   }
 
   // list permissions for given user
@@ -510,7 +432,8 @@ public class SentryConfigTool {
   }
 
   /**
-   *  parse arguments
+   * parse arguments
+   * 
    * <pre>
    *   -d,--debug                  Enable debug output
    *   -e,--query <arg>            Query privilege verification, requires -u
@@ -523,7 +446,10 @@ public class SentryConfigTool {
    *   -u,--user <arg>             user name
    *   -v,--validate               Validate policy file
    *   -I,--import                 Import policy file
+   *   -E,--export                 Export policy file
+   *   -o,--overwrite              Overwrite the exist role data when do the import
    * </pre>
+   * 
    * @param args
    */
   private void parseArgs(String[] args) {
@@ -549,9 +475,12 @@ public class SentryConfigTool {
         "list privileges for given user, requires -u");
     listPrivsOpt.setRequired(false);
 
-    Option importOpt = new Option("I", "import", false,
+    Option importOpt = new Option("I", "import", true,
         "Import policy file");
+    importOpt.setRequired(false);
 
+    Option exportOpt = new Option("E", "export", true, "Export policy file");
+    exportOpt.setRequired(false);
     // required args
     OptionGroup sentryOptGroup = new OptionGroup();
     sentryOptGroup.addOption(helpOpt);
@@ -560,6 +489,7 @@ public class SentryConfigTool {
     sentryOptGroup.addOption(listPermsOpt);
     sentryOptGroup.addOption(listPrivsOpt);
     sentryOptGroup.addOption(importOpt);
+    sentryOptGroup.addOption(exportOpt);
     sentryOptGroup.setRequired(true);
     sentryOptions.addOptionGroup(sentryOptGroup);
 
@@ -590,6 +520,10 @@ public class SentryConfigTool {
     debugOpt.setRequired(false);
     sentryOptions.addOption(debugOpt);
 
+    Option overwriteOpt = new Option("o", "overwrite", false, "enable import overwrite");
+    overwriteOpt.setRequired(false);
+    sentryOptions.addOption(overwriteOpt);
+
     try {
       Parser parser = new GnuParser();
       CommandLine cmd = parser.parse(sentryOptions, args);
@@ -612,11 +546,15 @@ public class SentryConfigTool {
         } else if (opt.getOpt().equals("v")) {
           setValidate(true);
         } else if (opt.getOpt().equals("I")) {
-          setImportPolicy(true);
+          setImportPolicyFilePath(opt.getValue());
+        } else if (opt.getOpt().equals("E")) {
+          setExportPolicyFilePath(opt.getValue());
         } else if (opt.getOpt().equals("h")) {
           usage(sentryOptions);
         } else if (opt.getOpt().equals("d")) {
           enableDebug = true;
+        } else if (opt.getOpt().equals("o")) {
+          setImportOverwriteRole(true);
         }
       }
 
@@ -653,10 +591,14 @@ public class SentryConfigTool {
           sentryTool.validatePolicy();
         }
 
-        if (sentryTool.isImportPolicy()) {
+        if (!StringUtils.isEmpty(sentryTool.getImportPolicyFilePath())) {
           sentryTool.importPolicy();
         }
 
+        if (!StringUtils.isEmpty(sentryTool.getExportPolicyFilePath())) {
+          sentryTool.exportPolicy();
+        }
+
         // list permissions for give user
         if (sentryTool.isListPrivs()) {
           sentryTool.listPrivs();

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92cde111/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
index f31fa54..4f87d5a 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
@@ -16,18 +16,17 @@
  */
 package org.apache.sentry.binding.hive.conf;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 
 public class HiveAuthzConf extends Configuration {
 
@@ -61,6 +60,9 @@ public class HiveAuthzConf extends Configuration {
     AUTHZ_PROVIDER_RESOURCE("sentry.hive.provider.resource", ""),
     AUTHZ_PROVIDER_BACKEND("sentry.hive.provider.backend", "org.apache.sentry.provider.file.SimpleFileProviderBackend"),
     AUTHZ_POLICY_ENGINE("sentry.hive.policy.engine", "org.apache.sentry.policy.db.SimpleDBPolicyEngine"),
+    AUTHZ_POLICY_FILE_FORMATTER(
+        "sentry.hive.policy.file.formatter",
+        "org.apache.sentry.binding.hive.SentryIniPolicyFileFormatter"),
     AUTHZ_SERVER_NAME("sentry.hive.server", "HS2"),
     AUTHZ_RESTRICT_DEFAULT_DB("sentry.hive.restrict.defaultDB", "false"),
     SENTRY_TESTING_MODE("sentry.hive.testing.mode", "false"),

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92cde111/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryIniPolicyFileFormatter.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryIniPolicyFileFormatter.java b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryIniPolicyFileFormatter.java
new file mode 100644
index 0000000..655417b
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryIniPolicyFileFormatter.java
@@ -0,0 +1,220 @@
+/*
+ * 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.binding.hive;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertTrue;
+
+import java.io.File;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.provider.common.PolicyFileConstants;
+import org.apache.sentry.provider.common.ProviderConstants;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.common.io.Files;
+import com.google.common.io.Resources;
+
+public class TestSentryIniPolicyFileFormatter {
+
+  private static final String RESOURCE_PATH = "testImportExportPolicy.ini";
+  // define the privileges
+  public static String PRIVILIEGE1 = "server=server1";
+  public static String PRIVILIEGE2 = "server=server1->action=select->grantoption=false";
+  public static String PRIVILIEGE3 = "server=server1->db=db2->action=insert->grantoption=true";
+  public static String PRIVILIEGE4 = "server=server1->db=db1->table=tbl1->action=insert";
+  public static String PRIVILIEGE5 = "server=server1->db=db1->table=tbl2->column=col1->action=insert";
+  public static String PRIVILIEGE6 = "server=server1->db=db1->table=tbl3->column=col1->action=*->grantoption=true";
+  public static String PRIVILIEGE7 = "server=server1->db=db1->table=tbl4->column=col1->action=all->grantoption=true";
+  public static String PRIVILIEGE8 = "server=server1->uri=hdfs://testserver:9999/path2->action=insert";
+
+  private Map<String, Map<String, Set<String>>> policyFileMappingData1;
+  private Map<String, Map<String, Set<String>>> policyFileMappingData2;
+  private Map<String, Map<String, Set<String>>> policyFileMappingData3;
+  private Map<String, Map<String, Set<String>>> policyFileMappingData4;
+  private Map<String, Map<String, Set<String>>> policyFileMappingData5;
+
+  private void prepareTestData() {
+    // test data for:
+    // [groups]
+    // group1=role1,role2,role3
+    // group2=role1,role2,role3
+    // group3=role1,role2,role3
+    // [roles]
+    // role1=privilege1,privilege2,privilege3,privilege4,privilege5,privilege6,privilege7,privilege8
+    // role2=privilege1,privilege2,privilege3,privilege4,privilege5,privilege6,privilege7,privilege8
+    // role3=privilege1,privilege2,privilege3,privilege4,privilege5,privilege6,privilege7,privilege8
+    policyFileMappingData1 = Maps.newHashMap();
+    Map<String, Set<String>> groupRolesMap = Maps.newHashMap();
+    Map<String, Set<String>> rolePrivilegesMap = Maps.newHashMap();
+    Set<String> roles = Sets.newHashSet("role1", "role2", "role3");
+    groupRolesMap.put("group1", roles);
+    groupRolesMap.put("group2", roles);
+    groupRolesMap.put("group3", roles);
+    for (String roleName : roles) {
+      rolePrivilegesMap.put(roleName, Sets.newHashSet(PRIVILIEGE1, PRIVILIEGE2, PRIVILIEGE3,
+          PRIVILIEGE4, PRIVILIEGE5, PRIVILIEGE6, PRIVILIEGE7, PRIVILIEGE8));
+    }
+    policyFileMappingData1.put(PolicyFileConstants.GROUPS, groupRolesMap);
+    policyFileMappingData1.put(PolicyFileConstants.ROLES, rolePrivilegesMap);
+
+    // test data for:
+    // [groups]
+    // group1=role1
+    // group2=role2
+    // group3=role3
+    // [roles]
+    // role1=privilege1,privilege2,privilege3
+    // role2=privilege4,privilege5,privilege6
+    // role3=privilege7,privilege8
+    policyFileMappingData2 = Maps.newHashMap();
+    groupRolesMap = Maps.newHashMap();
+    rolePrivilegesMap = Maps.newHashMap();
+    groupRolesMap.put("group1", Sets.newHashSet("role1"));
+    groupRolesMap.put("group2", Sets.newHashSet("role2"));
+    groupRolesMap.put("group3", Sets.newHashSet("role3"));
+    rolePrivilegesMap.put("role1", Sets.newHashSet(PRIVILIEGE1, PRIVILIEGE2, PRIVILIEGE3));
+    rolePrivilegesMap.put("role2", Sets.newHashSet(PRIVILIEGE4, PRIVILIEGE5, PRIVILIEGE6));
+    rolePrivilegesMap.put("role3", Sets.newHashSet(PRIVILIEGE7, PRIVILIEGE8));
+    policyFileMappingData2.put(PolicyFileConstants.GROUPS, groupRolesMap);
+    policyFileMappingData2.put(PolicyFileConstants.ROLES, rolePrivilegesMap);
+
+    // test data for:
+    // [groups]
+    // group1=role1,role2
+    // group2=role1,role2,role3
+    // group3=role2,role3
+    // [roles]
+    // role1=privilege1,privilege2,privilege3,privilege4
+    // role2=privilege3,privilege4,privilege5,privilege6
+    // role3=privilege5,privilege6,privilege7,privilege8
+    policyFileMappingData3 = Maps.newHashMap();
+    groupRolesMap = Maps.newHashMap();
+    rolePrivilegesMap = Maps.newHashMap();
+    groupRolesMap.put("group1", Sets.newHashSet("role1", "role2"));
+    groupRolesMap.put("group2", Sets.newHashSet("role1", "role2", "role3"));
+    groupRolesMap.put("group3", Sets.newHashSet("role2", "role3"));
+    rolePrivilegesMap.put("role1",
+        Sets.newHashSet(PRIVILIEGE1, PRIVILIEGE2, PRIVILIEGE3, PRIVILIEGE4));
+    rolePrivilegesMap.put("role2",
+        Sets.newHashSet(PRIVILIEGE3, PRIVILIEGE4, PRIVILIEGE5, PRIVILIEGE6));
+    rolePrivilegesMap.put("role3",
+        Sets.newHashSet(PRIVILIEGE5, PRIVILIEGE6, PRIVILIEGE7, PRIVILIEGE8));
+    policyFileMappingData3.put(PolicyFileConstants.GROUPS, groupRolesMap);
+    policyFileMappingData3.put(PolicyFileConstants.ROLES, rolePrivilegesMap);
+
+    // test data for groups only
+    policyFileMappingData4 = Maps.newHashMap();
+    groupRolesMap = Maps.newHashMap();
+    rolePrivilegesMap = Maps.newHashMap();
+    groupRolesMap.put("group1", Sets.newHashSet("role1", "role2"));
+    policyFileMappingData4.put(PolicyFileConstants.GROUPS, groupRolesMap);
+    policyFileMappingData4.put(PolicyFileConstants.ROLES, rolePrivilegesMap);
+
+    // test empty data
+    policyFileMappingData5 = Maps.newHashMap();
+    groupRolesMap = Maps.newHashMap();
+    rolePrivilegesMap = Maps.newHashMap();
+    policyFileMappingData5.put(PolicyFileConstants.GROUPS, groupRolesMap);
+    policyFileMappingData5.put(PolicyFileConstants.ROLES, rolePrivilegesMap);
+  }
+
+  @Test
+  public void testImportExport() throws Exception {
+    prepareTestData();
+    File baseDir = Files.createTempDir();
+    String resourcePath = (new File(baseDir, RESOURCE_PATH)).getAbsolutePath();
+    HiveAuthzConf authzConf = new HiveAuthzConf(Resources.getResource("sentry-site.xml"));
+    SentryIniPolicyFileFormatter iniFormatter = new SentryIniPolicyFileFormatter();
+
+    // test data1
+    iniFormatter.write(resourcePath, policyFileMappingData1);
+    Map<String, Map<String, Set<String>>> parsedMappingData = iniFormatter.parse(resourcePath,
+        authzConf);
+    validateSentryMappingData(parsedMappingData, policyFileMappingData1);
+
+    // test data2
+    iniFormatter.write(resourcePath, policyFileMappingData2);
+    parsedMappingData = iniFormatter.parse(resourcePath, authzConf);
+    validateSentryMappingData(parsedMappingData, policyFileMappingData2);
+
+    // test data3
+    iniFormatter.write(resourcePath, policyFileMappingData3);
+    parsedMappingData = iniFormatter.parse(resourcePath, authzConf);
+    validateSentryMappingData(parsedMappingData, policyFileMappingData3);
+
+    // test data4
+    iniFormatter.write(resourcePath, policyFileMappingData4);
+    parsedMappingData = iniFormatter.parse(resourcePath, authzConf);
+    assertTrue(parsedMappingData.get(PolicyFileConstants.GROUPS).isEmpty());
+    assertTrue(parsedMappingData.get(PolicyFileConstants.ROLES).isEmpty());
+
+    // test data5
+    iniFormatter.write(resourcePath, policyFileMappingData5);
+    parsedMappingData = iniFormatter.parse(resourcePath, authzConf);
+    assertTrue(parsedMappingData.get(PolicyFileConstants.GROUPS).isEmpty());
+    assertTrue(parsedMappingData.get(PolicyFileConstants.ROLES).isEmpty());
+    (new File(baseDir, RESOURCE_PATH)).delete();
+  }
+
+  // verify the mapping data
+  public void validateSentryMappingData(Map<String, Map<String, Set<String>>> actualMappingData,
+      Map<String, Map<String, Set<String>>> expectedMappingData) {
+    validateGroupRolesMap(actualMappingData.get(PolicyFileConstants.GROUPS),
+        expectedMappingData.get(PolicyFileConstants.GROUPS));
+    validateRolePrivilegesMap(actualMappingData.get(PolicyFileConstants.ROLES),
+        expectedMappingData.get(PolicyFileConstants.ROLES));
+  }
+
+  // verify the mapping data for [group,role]
+  private void validateGroupRolesMap(Map<String, Set<String>> actualMap,
+      Map<String, Set<String>> expectedMap) {
+    assertEquals(expectedMap.keySet().size(), actualMap.keySet().size());
+    for (String groupName : actualMap.keySet()) {
+      Set<String> actualRoles = actualMap.get(groupName);
+      Set<String> expectedRoles = expectedMap.get(groupName);
+      assertEquals(actualRoles.size(), expectedRoles.size());
+      assertTrue(actualRoles.equals(expectedRoles));
+    }
+  }
+
+  // verify the mapping data for [role,privilege]
+  private void validateRolePrivilegesMap(Map<String, Set<String>> actualMap,
+      Map<String, Set<String>> expectedMap) {
+    assertEquals(expectedMap.keySet().size(), actualMap.keySet().size());
+    for (String roleName : actualMap.keySet()) {
+      Set<String> actualPrivileges = actualMap.get(roleName);
+      Set<String> exceptedPrivileges = expectedMap.get(roleName);
+      assertEquals(exceptedPrivileges.size(), actualPrivileges.size());
+      for (String actualPrivilege : actualPrivileges) {
+        boolean isFound = exceptedPrivileges.contains(actualPrivilege);
+        if (!isFound) {
+          String withOptionPrivilege = ProviderConstants.AUTHORIZABLE_JOINER.join(actualPrivilege,
+              ProviderConstants.KV_JOINER.join(PolicyFileConstants.PRIVILEGE_GRANT_OPTION_NAME,
+                  "false"));
+          isFound = exceptedPrivileges.contains(withOptionPrivilege);
+        }
+        assertTrue(isFound);
+      }
+    }
+  }
+}